Merging r1566359 through r1568420 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1568437 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jing Zhao 2014-02-14 18:32:37 +00:00
commit ba4b10354c
163 changed files with 12303 additions and 3035 deletions

View File

@ -312,6 +312,11 @@ Release 2.4.0 - UNRELEASED
HADOOP-10295. Allow distcp to automatically identify the checksum type of
source files and use it for the target. (jing9 and Laurent Goujon)
HADOOP-10333. Fix grammatical error in overview.html document.
(René Nyffenegger via suresh)
HADOOP-10343. Change info to debug log in LossyRetryInvocationHandler. (arpit)
OPTIMIZATIONS
BUG FIXES
@ -328,15 +333,36 @@ Release 2.4.0 - UNRELEASED
HADOOP-10330. TestFrameDecoder fails if it cannot bind port 12345.
(Arpit Agarwal)
Release 2.3.0 - UNRELEASED
HADOOP-10326. M/R jobs can not access S3 if Kerberos is enabled. (bc Wong
via atm)
HADOOP-10338. Cannot get the FileStatus of the root inode from the new
Globber (cmccabe)
HADOOP-10249. LdapGroupsMapping should trim ldap password read from file.
(Dilli Armugam via suresh)
Release 2.3.1 - UNRELEASED
INCOMPATIBLE CHANGES
HADOOP-8545. Filesystem Implementation for OpenStack Swift
(Dmitry Mezhensky, David Dobbins, Stevel via stevel)
NEW FEATURES
IMPROVEMENTS
OPTIMIZATIONS
BUG FIXES
Release 2.3.0 - 2014-02-18
INCOMPATIBLE CHANGES
NEW FEATURES
HADOOP-8545. Filesystem Implementation for OpenStack Swift
(Dmitry Mezhensky, David Dobbins, Stevel via stevel)
IMPROVEMENTS
HADOOP-10046. Print a log message when SSL is enabled.

View File

@ -114,7 +114,8 @@ private String schemeFromPath(Path path) throws IOException {
if (fs != null) {
scheme = fs.getUri().getScheme();
} else {
scheme = fc.getDefaultFileSystem().getUri().getScheme();
scheme = fc.getFSofPath(fc.fixRelativePart(path)).
getUri().getScheme();
}
}
return scheme;
@ -126,7 +127,8 @@ private String authorityFromPath(Path path) throws IOException {
if (fs != null) {
authority = fs.getUri().getAuthority();
} else {
authority = fc.getDefaultFileSystem().getUri().getAuthority();
authority = fc.getFSofPath(fc.fixRelativePart(path)).
getUri().getAuthority();
}
}
return authority ;
@ -162,18 +164,26 @@ public FileStatus[] glob() throws IOException {
// Starting out at the root of the filesystem, we try to match
// filesystem entries against pattern components.
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()
&& Path.isWindowsAbsolutePath(absPattern.toUri().getPath(), true)) {
// 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
// root of the filesystem on that drive.
String driveLetter = components.remove(0);
candidates.add(new FileStatus(0, true, 0, 0, 0, new Path(scheme,
authority, Path.SEPARATOR + driveLetter + Path.SEPARATOR)));
rootPlaceholder = new FileStatus(0, true, 0, 0, 0, new Path(scheme,
authority, Path.SEPARATOR + driveLetter + Path.SEPARATOR));
} else {
candidates.add(new FileStatus(0, true, 0, 0, 0,
new Path(scheme, authority, Path.SEPARATOR)));
rootPlaceholder = new FileStatus(0, true, 0, 0, 0,
new Path(scheme, authority, Path.SEPARATOR));
}
candidates.add(rootPlaceholder);
for (int componentIdx = 0; componentIdx < components.size();
componentIdx++) {
@ -245,6 +255,12 @@ public FileStatus[] glob() throws IOException {
candidates = newCandidates;
}
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
// end, once the full path is built up.
if (filter.accept(status.getPath())) {

View File

@ -443,6 +443,12 @@ public long getDefaultBlockSize() {
return getConf().getLong("fs.s3.block.size", 64 * 1024 * 1024);
}
@Override
public String getCanonicalServiceName() {
// Does not support Token
return null;
}
// diagnostic methods
void dump() throws IOException {

View File

@ -733,4 +733,10 @@ public void setWorkingDirectory(Path newDir) {
public Path getWorkingDirectory() {
return workingDir;
}
@Override
public String getCanonicalServiceName() {
// Does not support Token
return null;
}
}

View File

@ -51,11 +51,15 @@ protected Object invokeMethod(Method method, Object[] args) throws Throwable {
int retryCount = RetryCount.get();
if (retryCount < this.numToDrop) {
RetryCount.set(++retryCount);
LOG.info("Drop the response. Current retryCount == " + retryCount);
if (LOG.isDebugEnabled()) {
LOG.debug("Drop the response. Current retryCount == " + retryCount);
}
throw new RetriableException("Fake Exception");
} else {
LOG.info("retryCount == " + retryCount
if (LOG.isDebugEnabled()) {
LOG.debug("retryCount == " + retryCount
+ ". It's time to normally process the response");
}
return result;
}
}

View File

@ -220,7 +220,7 @@ final public static void setPingInterval(Configuration conf, int pingInterval) {
* @param conf Configuration
* @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,
CommonConfigurationKeys.IPC_PING_INTERVAL_DEFAULT);
}

View File

@ -66,6 +66,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configuration.IntegerRanges;
@ -454,9 +455,10 @@ public void refreshServiceAcl(Configuration conf, PolicyProvider provider) {
* Refresh the service authorization ACL for the service handled by this server
* using the specified Configuration.
*/
public void refreshServiceAclWithConfigration(Configuration conf,
@Private
public void refreshServiceAclWithLoadedConfiguration(Configuration conf,
PolicyProvider provider) {
serviceAuthorizationManager.refreshWithConfiguration(conf, provider);
serviceAuthorizationManager.refreshWithLoadedConfiguration(conf, provider);
}
/**
* Returns a handle to the serviceAuthorizationManager (required in tests)

View File

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

View File

@ -27,6 +27,7 @@
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
@ -240,4 +241,18 @@ public static synchronized Groups getUserToGroupsMappingService(
}
return GROUPS;
}
/**
* Create new groups used to map user-to-groups with loaded configuration.
* @param conf
* @return the groups being used to map user-to-groups.
*/
@Private
public static synchronized Groups
getUserToGroupsMappingServiceWithLoadedConfiguration(
Configuration conf) {
GROUPS = new Groups(conf);
return GROUPS;
}
}

View File

@ -356,7 +356,7 @@ String extractPassword(String pwFile) {
c = reader.read();
}
reader.close();
return password.toString();
return password.toString().trim();
} catch (IOException ioe) {
throw new RuntimeException("Could not read password file: " + pwFile, ioe);
}

View File

@ -26,6 +26,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
@ -122,10 +123,11 @@ public synchronized void refresh(Configuration conf,
// Make a copy of the original config, and load the policy file
Configuration policyConf = new Configuration(conf);
policyConf.addResource(policyFile);
refreshWithConfiguration(policyConf, provider);
refreshWithLoadedConfiguration(policyConf, provider);
}
public synchronized void refreshWithConfiguration(Configuration conf,
@Private
public synchronized void refreshWithLoadedConfiguration(Configuration conf,
PolicyProvider provider) {
final Map<Class<?>, AccessControlList> newAcls =
new IdentityHashMap<Class<?>, AccessControlList>();

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

@ -57,7 +57,7 @@ <h3>Platforms</h3>
<ul>
<li>
Hadoop was been demonstrated on GNU/Linux clusters with 2000 nodes.
Hadoop has been demonstrated on GNU/Linux clusters with more than 4000 nodes.
</li>
<li>
Windows is also a supported platform.

View File

@ -54,5 +54,10 @@ public void testBlockSize() throws Exception {
assertEquals("Double default block size", newBlockSize,
fs.getFileStatus(file).getBlockSize());
}
public void testCanonicalName() throws Exception {
assertNull("s3 doesn't support security token and shouldn't have canonical name",
fs.getCanonicalServiceName());
}
}

View File

@ -48,7 +48,12 @@ protected void tearDown() throws Exception {
store.purge("test");
super.tearDown();
}
public void testCanonicalName() throws Exception {
assertNull("s3n doesn't support security token and shouldn't have canonical name",
fs.getCanonicalServiceName());
}
public void testListStatusForRoot() throws Exception {
FileStatus[] paths = fs.listStatus(path("/"));
assertEquals("Root directory is not empty; ", 0, paths.length);
@ -60,7 +65,7 @@ public void testListStatusForRoot() throws Exception {
assertEquals(1, paths.length);
assertEquals(path("/test"), paths[0].getPath());
}
public void testNoTrailingBackslashOnBucket() throws Exception {
assertTrue(fs.getFileStatus(new Path(fs.getUri().toString())).isDirectory());
}

View File

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

View File

@ -545,7 +545,8 @@ public READLINK3Response readlink(XDR xdr, SecurityHandler securityHandler,
return new READLINK3Response(Nfs3Status.NFS3ERR_SERVERFAULT);
}
if (MAX_READ_TRANSFER_SIZE < target.getBytes().length) {
return new READLINK3Response(Nfs3Status.NFS3ERR_IO, postOpAttr, null);
return new READLINK3Response(Nfs3Status.NFS3ERR_IO, postOpAttr,
new byte[0]);
}
return new READLINK3Response(Nfs3Status.NFS3_OK, postOpAttr,
@ -1828,7 +1829,8 @@ public COMMIT3Response commit(XDR xdr, Channel channel, int xid,
} catch (IOException e1) {
LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId());
}
WccData fileWcc = new WccData(Nfs3Utils.getWccAttr(preOpAttr), postOpAttr);
WccData fileWcc = new WccData(preOpAttr == null ? null
: Nfs3Utils.getWccAttr(preOpAttr), postOpAttr);
return new COMMIT3Response(Nfs3Status.NFS3ERR_IO, fileWcc,
Nfs3Constant.WRITE_COMMIT_VERF);
}

View File

@ -120,31 +120,6 @@ Trunk (Unreleased)
HDFS-5041. Add the time of last heartbeat to dead server Web UI (Shinichi
Yamashita via brandonli)
HDFS-5531. Combine the getNsQuota() and getDsQuota() methods in INode.
(szetszwo)
HDFS-5285. Flatten INodeFile hierarchy: Replace INodeFileUnderConstruction
and INodeFileUnderConstructionWithSnapshot with FileUnderContructionFeature.
(jing9 via szetszwo)
HDFS-5286. Flatten INodeDirectory hierarchy: Replace INodeDirectoryWithQuota
with DirectoryWithQuotaFeature. (szetszwo)
HDFS-5537. Remove FileWithSnapshot interface. (jing9 via szetszwo)
HDFS-5554. Flatten INodeFile hierarchy: Replace INodeFileWithSnapshot with
FileWithSnapshotFeature. (jing9 via szetszwo)
HDFS-5647. Merge INodeDirectory.Feature and INodeFile.Feature. (Haohui Mai
via jing9)
HDFS-5632. Flatten INodeDirectory hierarchy: Replace
INodeDirectoryWithSnapshot with DirectoryWithSnapshotFeature.
(jing9 via szetszwo)
HDFS-5715. Use Snapshot ID to indicate the corresponding Snapshot for a
FileDiff/DirectoryDiff. (jing9)
HDFS-5721. sharedEditsImage in Namenode#initializeSharedEdits() should be
closed before method returns. (Ted Yu via junping_du)
@ -275,8 +250,6 @@ Trunk (Unreleased)
HDFS-5719. FSImage#doRollback() should close prevState before return
(Ted Yu via brandonli)
HDFS-5726. Fix compilation error in AbstractINodeDiff for JDK7. (jing9)
HDFS-5768. Consolidate the serialization code in DelegationTokenSecretManager
(Haohui Mai via brandonli)
@ -286,6 +259,60 @@ Trunk (Unreleased)
HDFS-5794. Fix the inconsistency of layout version number of
ADD_DATANODE_AND_STORAGE_UUIDS between trunk and branch-2. (jing9)
BREAKDOWN OF HDFS-5698 SUBTASKS AND RELATED JIRAS
HDFS-5717. Save FSImage header in protobuf. (Haohui Mai via jing9)
HDFS-5738. Serialize INode information in protobuf. (Haohui Mai via jing9)
HDFS-5772. Serialize under-construction file information in FSImage. (jing9)
HDFS-5783. Compute the digest before loading FSImage. (Haohui Mai via jing9)
HDFS-5785. Serialize symlink in protobuf. (Haohui Mai via jing9)
HDFS-5793. Optimize the serialization of PermissionStatus. (Haohui Mai via
jing9)
HDFS-5743. Use protobuf to serialize snapshot information. (jing9)
HDFS-5774. Serialize CachePool directives in protobuf. (Haohui Mai via jing9)
HDFS-5744. Serialize information for token managers in protobuf. (Haohui Mai
via jing9)
HDFS-5824. Add a Type field in Snapshot DiffEntry's protobuf definition.
(jing9)
HDFS-5808. Implement cancellation when saving FSImage. (Haohui Mai via jing9)
HDFS-5826. Update the stored edit logs to be consistent with the changes in
HDFS-5698 branch. (Haohui Mai via jing9)
HDFS-5797. Implement offline image viewer. (Haohui Mai via jing9)
HDFS-5771. Track progress when loading fsimage. (Haohui Mai via cnauroth)
HDFS-5871. Use PBHelper to serialize CacheDirectiveInfoExpirationProto.
(Haohui Mai via jing9)
HDFS-5884. LoadDelegator should use IOUtils.readFully() to read the magic
header. (Haohui Mai via jing9)
HDFS-5885. Add annotation for repeated fields in the protobuf definition.
(Haohui Mai via jing9)
HDFS-5906. Fixing findbugs and javadoc warnings in the HDFS-5698 branch.
(Haohui Mai via jing9)
HDFS-5911. The id of a CacheDirective instance does not get serialized in
the protobuf-fsimage. (Haohui Mai via jing9)
HDFS-5915. Refactor FSImageFormatProtobuf to simplify cross section reads.
(Haohui Mai via cnauroth)
HDFS-5847. Consolidate INodeReference into a separate section. (jing9)
Release 2.4.0 - UNRELEASED
INCOMPATIBLE CHANGES
@ -311,6 +338,44 @@ Release 2.4.0 - UNRELEASED
HDFS-4911. Reduce PeerCache timeout to be commensurate with
dfs.datanode.socket.reuse.keepalive (cmccabe)
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)
HDFS-5531. Combine the getNsQuota() and getDsQuota() methods in INode.
(szetszwo)
HDFS-5285. Flatten INodeFile hierarchy: Replace INodeFileUnderConstruction
and INodeFileUnderConstructionWithSnapshot with FileUnderContructionFeature.
(jing9 via szetszwo)
HDFS-5286. Flatten INodeDirectory hierarchy: Replace INodeDirectoryWithQuota
with DirectoryWithQuotaFeature. (szetszwo)
HDFS-5537. Remove FileWithSnapshot interface. (jing9 via szetszwo)
HDFS-5554. Flatten INodeFile hierarchy: Replace INodeFileWithSnapshot with
FileWithSnapshotFeature. (jing9 via szetszwo)
HDFS-5647. Merge INodeDirectory.Feature and INodeFile.Feature. (Haohui Mai
via jing9)
HDFS-5632. Flatten INodeDirectory hierarchy: Replace
INodeDirectoryWithSnapshot with DirectoryWithSnapshotFeature.
(jing9 via szetszwo)
HDFS-5715. Use Snapshot ID to indicate the corresponding Snapshot for a
FileDiff/DirectoryDiff. (jing9)
HDFS-5726. Fix compilation error in AbstractINodeDiff for JDK7. (jing9)
OPTIMIZATIONS
HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery
@ -353,7 +418,52 @@ Release 2.4.0 - UNRELEASED
HDFS-5900. Cannot set cache pool limit of "unlimited" via CacheAdmin.
(wang)
Release 2.3.0 - UNRELEASED
HDFS-5886. Potential null pointer deference in RpcProgramNfs3#readlink()
(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)
HDFS-5941. add dfs.namenode.secondary.https-address and
dfs.namenode.secondary.https-address in hdfs-default.xml.
(Haohui Mai via cnauroth)
HDFS-5913. Nfs3Utils#getWccAttr() should check attr parameter against null
(brandonli)
HDFS-5934. New Namenode UI back button doesn't work as expected
(Travis Thompson via brandonli)
HDFS-5901. NameNode new UI doesn't support IE8 and IE9 on windows 7
(Vinayakumar B via brandonli)
HDFS-5943. 'dfs.namenode.https-address' property is not loaded from
configuration in federation setup. (suresh)
Release 2.3.1 - UNRELEASED
INCOMPATIBLE CHANGES
NEW FEATURES
IMPROVEMENTS
OPTIMIZATIONS
BUG FIXES
Release 2.3.0 - 2014-02-18
INCOMPATIBLE CHANGES
@ -891,6 +1001,12 @@ Release 2.3.0 - UNRELEASED
HDFS-5873. dfs.http.policy should have higher precedence over dfs.https.enable.
(Haohui Mai via jing9)
HDFS-5837. dfs.namenode.replication.considerLoad should consider
decommissioned nodes. (Tao Luo via shv)
HDFS-5921. Cannot browse file system via NN web UI if any directory has
the sticky bit set. (atm)
BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
HDFS-4985. Add storage type to the protocol and expose it in block report

View File

@ -8,6 +8,9 @@
<Match>
<Package name="org.apache.hadoop.hdfs.server.namenode.ha.proto" />
</Match>
<Match>
<Class name="~org.apache.hadoop.hdfs.server.namenode.FsImageProto.*" />
</Match>
<Match>
<Package name="org.apache.hadoop.hdfs.qjournal.protocol" />
</Match>

View File

@ -458,6 +458,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
<includes>
<include>ClientDatanodeProtocol.proto</include>
<include>DatanodeProtocol.proto</include>
<include>fsimage.proto</include>
</includes>
</source>
<output>${project.build.directory}/generated-sources/java</output>

View File

@ -139,7 +139,7 @@ elif [ "$COMMAND" = "balancer" ] ; then
elif [ "$COMMAND" = "jmxget" ] ; then
CLASS=org.apache.hadoop.hdfs.tools.JMXGet
elif [ "$COMMAND" = "oiv" ] ; then
CLASS=org.apache.hadoop.hdfs.tools.offlineImageViewer.OfflineImageViewer
CLASS=org.apache.hadoop.hdfs.tools.offlineImageViewer.OfflineImageViewerPB
elif [ "$COMMAND" = "oev" ] ; then
CLASS=org.apache.hadoop.hdfs.tools.offlineEditsViewer.OfflineEditsViewer
elif [ "$COMMAND" = "fetchdt" ] ; then

View File

@ -23,7 +23,6 @@
import org.apache.hadoop.fs.ByteBufferReadable;
import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.hdfs.client.ClientMmap;
import org.apache.hadoop.hdfs.client.ClientMmapManager;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
/**
@ -97,6 +96,5 @@ public interface BlockReader extends ByteBufferReadable {
* @return The ClientMmap object, or null if mmap is not
* supported.
*/
ClientMmap getClientMmap(EnumSet<ReadOption> opts,
ClientMmapManager mmapManager);
ClientMmap getClientMmap(EnumSet<ReadOption> opts);
}

View File

@ -24,217 +24,748 @@
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
import org.apache.hadoop.hdfs.client.ShortCircuitCache.ShortCircuitReplicaCreator;
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo;
import org.apache.hadoop.hdfs.net.DomainPeer;
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
/**
* Utility class to create BlockReader implementations.
*/
@InterfaceAudience.Private
public class BlockReaderFactory {
/**
* Create a new BlockReader specifically to satisfy a read.
* This method also sends the OP_READ_BLOCK request.
*
* @param conf the DFSClient configuration
* @param file File location
* @param block The block object
* @param blockToken The block token for security
* @param startOffset The read offset, relative to block head
* @param len The number of bytes to read, or -1 to read as many as
* possible.
* @param bufferSize The IO buffer size (not the client buffer size)
* Ignored except on the legacy BlockReader.
* @param verifyChecksum Whether to verify checksum
* @param clientName Client name. Used for log messages.
* @param peer The peer
* @param datanodeID The datanode that the Peer is connected to
* @param domainSocketFactory The DomainSocketFactory to notify if the Peer
* is a DomainPeer which turns out to be faulty.
* If null, no factory will be notified in this
* case.
* @param allowShortCircuitLocalReads True if short-circuit local reads
* should be allowed.
* @return New BlockReader instance
*/
public static BlockReader newBlockReader(DFSClient.Conf conf,
String file,
ExtendedBlock block,
Token<BlockTokenIdentifier> blockToken,
long startOffset, long len,
boolean verifyChecksum,
String clientName,
Peer peer,
DatanodeID datanodeID,
DomainSocketFactory domSockFactory,
PeerCache peerCache,
FileInputStreamCache fisCache,
boolean allowShortCircuitLocalReads,
CachingStrategy cachingStrategy)
throws IOException {
peer.setReadTimeout(conf.socketTimeout);
peer.setWriteTimeout(HdfsServerConstants.WRITE_TIMEOUT);
public class BlockReaderFactory implements ShortCircuitReplicaCreator {
static final Log LOG = LogFactory.getLog(BlockReaderFactory.class);
if (peer.getDomainSocket() != null) {
if (allowShortCircuitLocalReads && !conf.useLegacyBlockReaderLocal) {
// If this is a domain socket, and short-circuit local reads are
// enabled, try to set up a BlockReaderLocal.
BlockReader reader = newShortCircuitBlockReader(conf, file,
block, blockToken, startOffset, len, peer, datanodeID,
domSockFactory, verifyChecksum, fisCache, cachingStrategy);
@VisibleForTesting
static ShortCircuitReplicaCreator
createShortCircuitReplicaInfoCallback = null;
private final DFSClient.Conf conf;
/**
* The file name, for logging and debugging purposes.
*/
private String fileName;
/**
* The block ID and block pool ID to use.
*/
private ExtendedBlock block;
/**
* The block token to use for security purposes.
*/
private Token<BlockTokenIdentifier> token;
/**
* The offset within the block to start reading at.
*/
private long startOffset;
/**
* If false, we won't try to verify the block checksum.
*/
private boolean verifyChecksum;
/**
* The name of this client.
*/
private String clientName;
/**
* The DataNode we're talking to.
*/
private DatanodeInfo datanode;
/**
* If false, we won't try short-circuit local reads.
*/
private boolean allowShortCircuitLocalReads;
/**
* The ClientContext to use for things like the PeerCache.
*/
private ClientContext clientContext;
/**
* Number of bytes to read. -1 indicates no limit.
*/
private long length = -1;
/**
* Caching strategy to use when reading the block.
*/
private CachingStrategy cachingStrategy;
/**
* Socket address to use to connect to peer.
*/
private InetSocketAddress inetSocketAddress;
/**
* Remote peer factory to use to create a peer, if needed.
*/
private RemotePeerFactory remotePeerFactory;
/**
* UserGroupInformation to use for legacy block reader local objects, if needed.
*/
private UserGroupInformation userGroupInformation;
/**
* Configuration to use for legacy block reader local objects, if needed.
*/
private Configuration configuration;
/**
* Information about the domain socket path we should use to connect to the
* local peer-- or null if we haven't examined the local domain socket.
*/
private DomainSocketFactory.PathInfo pathInfo;
/**
* The remaining number of times that we'll try to pull a socket out of the
* cache.
*/
private int remainingCacheTries;
public BlockReaderFactory(DFSClient.Conf conf) {
this.conf = conf;
this.remainingCacheTries = conf.nCachedConnRetry;
}
public BlockReaderFactory setFileName(String fileName) {
this.fileName = fileName;
return this;
}
public BlockReaderFactory setBlock(ExtendedBlock block) {
this.block = block;
return this;
}
public BlockReaderFactory setBlockToken(Token<BlockTokenIdentifier> token) {
this.token = token;
return this;
}
public BlockReaderFactory setStartOffset(long startOffset) {
this.startOffset = startOffset;
return this;
}
public BlockReaderFactory setVerifyChecksum(boolean verifyChecksum) {
this.verifyChecksum = verifyChecksum;
return this;
}
public BlockReaderFactory setClientName(String clientName) {
this.clientName = clientName;
return this;
}
public BlockReaderFactory setDatanodeInfo(DatanodeInfo datanode) {
this.datanode = datanode;
return this;
}
public BlockReaderFactory setAllowShortCircuitLocalReads(
boolean allowShortCircuitLocalReads) {
this.allowShortCircuitLocalReads = allowShortCircuitLocalReads;
return this;
}
public BlockReaderFactory setClientCacheContext(
ClientContext clientContext) {
this.clientContext = clientContext;
return this;
}
public BlockReaderFactory setLength(long length) {
this.length = length;
return this;
}
public BlockReaderFactory setCachingStrategy(
CachingStrategy cachingStrategy) {
this.cachingStrategy = cachingStrategy;
return this;
}
public BlockReaderFactory setInetSocketAddress (
InetSocketAddress inetSocketAddress) {
this.inetSocketAddress = inetSocketAddress;
return this;
}
public BlockReaderFactory setUserGroupInformation(
UserGroupInformation userGroupInformation) {
this.userGroupInformation = userGroupInformation;
return this;
}
public BlockReaderFactory setRemotePeerFactory(
RemotePeerFactory remotePeerFactory) {
this.remotePeerFactory = remotePeerFactory;
return this;
}
public BlockReaderFactory setConfiguration(
Configuration configuration) {
this.configuration = configuration;
return this;
}
/**
* Build a BlockReader with the given options.
*
* This function will do the best it can to create a block reader that meets
* all of our requirements. We prefer short-circuit block readers
* (BlockReaderLocal and BlockReaderLocalLegacy) over remote ones, since the
* former avoid the overhead of socket communication. If short-circuit is
* unavailable, our next fallback is data transfer over UNIX domain sockets,
* if dfs.client.domain.socket.data.traffic has been enabled. If that doesn't
* work, we will try to create a remote block reader that operates over TCP
* sockets.
*
* There are a few caches that are important here.
*
* The ShortCircuitCache stores file descriptor objects which have been passed
* from the DataNode.
*
* The DomainSocketFactory stores information about UNIX domain socket paths
* that we not been able to use in the past, so that we don't waste time
* retrying them over and over. (Like all the caches, it does have a timeout,
* though.)
*
* The PeerCache stores peers that we have used in the past. If we can reuse
* one of these peers, we avoid the overhead of re-opening a socket. However,
* if the socket has been timed out on the remote end, our attempt to reuse
* the socket may end with an IOException. For that reason, we limit our
* attempts at socket reuse to dfs.client.cached.conn.retry times. After
* that, we create new sockets. This avoids the problem where a thread tries
* to talk to a peer that it hasn't talked to in a while, and has to clean out
* every entry in a socket cache full of stale entries.
*
* @return The new BlockReader. We will not return null.
*
* @throws InvalidToken
* If the block token was invalid.
* InvalidEncryptionKeyException
* If the encryption key was invalid.
* Other IOException
* If there was another problem.
*/
public BlockReader build() throws IOException {
BlockReader reader = null;
Preconditions.checkNotNull(configuration);
if (conf.shortCircuitLocalReads && allowShortCircuitLocalReads) {
if (clientContext.getUseLegacyBlockReaderLocal()) {
reader = getLegacyBlockReaderLocal();
if (reader != null) {
// One we've constructed the short-circuit block reader, we don't
// need the socket any more. So let's return it to the cache.
if (peerCache != null) {
peerCache.put(datanodeID, peer);
} else {
IOUtils.cleanup(null, peer);
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": returning new legacy block reader local.");
}
return reader;
}
} else {
reader = getBlockReaderLocal();
if (reader != null) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": returning new block reader local.");
}
return reader;
}
}
// If this is a domain socket and we couldn't (or didn't want to) set
// up a BlockReaderLocal, check that we are allowed to pass data traffic
// over the socket before proceeding.
if (!conf.domainSocketDataTraffic) {
throw new IOException("Because we can't do short-circuit access, " +
"and data traffic over domain sockets is disabled, " +
"we cannot use this socket to talk to " + datanodeID);
}
if (conf.domainSocketDataTraffic) {
reader = getRemoteBlockReaderFromDomain();
if (reader != null) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": returning new remote block reader using " +
"UNIX domain socket on " + pathInfo.getPath());
}
return reader;
}
}
if (conf.useLegacyBlockReader) {
@SuppressWarnings("deprecation")
RemoteBlockReader reader = RemoteBlockReader.newBlockReader(file,
block, blockToken, startOffset, len, conf.ioBufferSize,
verifyChecksum, clientName, peer, datanodeID, peerCache,
cachingStrategy);
return reader;
} else {
return RemoteBlockReader2.newBlockReader(
file, block, blockToken, startOffset, len,
verifyChecksum, clientName, peer, datanodeID, peerCache,
cachingStrategy);
}
Preconditions.checkState(!DFSInputStream.tcpReadsDisabledForTesting,
"TCP reads were disabled for testing, but we failed to " +
"do a non-TCP read.");
return getRemoteBlockReaderFromTcp();
}
/**
* Create a new short-circuit BlockReader.
*
* Here, we ask the DataNode to pass us file descriptors over our
* DomainSocket. If the DataNode declines to do so, we'll return null here;
* otherwise, we'll return the BlockReaderLocal. If the DataNode declines,
* this function will inform the DomainSocketFactory that short-circuit local
* reads are disabled for this DataNode, so that we don't ask again.
*
* @param conf the configuration.
* @param file the file name. Used in log messages.
* @param block The block object.
* @param blockToken The block token for security.
* @param startOffset The read offset, relative to block head.
* @param len The number of bytes to read, or -1 to read
* as many as possible.
* @param peer The peer to use.
* @param datanodeID The datanode that the Peer is connected to.
* @param domSockFactory The DomainSocketFactory to notify if the Peer
* is a DomainPeer which turns out to be faulty.
* If null, no factory will be notified in this
* case.
* @param verifyChecksum True if we should verify the checksums.
* Note: even if this is true, when
* DFS_CLIENT_READ_CHECKSUM_SKIP_CHECKSUM_KEY is
* set or the block is mlocked, we will skip
* checksums.
*
* @return The BlockReaderLocal, or null if the
* DataNode declined to provide short-circuit
* access.
* @throws IOException If there was a communication error.
* Get {@link BlockReaderLocalLegacy} for short circuited local reads.
* This block reader implements the path-based style of local reads
* first introduced in HDFS-2246.
*/
private static BlockReaderLocal newShortCircuitBlockReader(
DFSClient.Conf conf, String file, ExtendedBlock block,
Token<BlockTokenIdentifier> blockToken, long startOffset,
long len, Peer peer, DatanodeID datanodeID,
DomainSocketFactory domSockFactory, boolean verifyChecksum,
FileInputStreamCache fisCache,
CachingStrategy cachingStrategy) throws IOException {
private BlockReader getLegacyBlockReaderLocal() throws IOException {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": trying to construct BlockReaderLocalLegacy");
}
if (!DFSClient.isLocalAddress(inetSocketAddress)) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": can't construct BlockReaderLocalLegacy because " +
"the address " + inetSocketAddress + " is not local");
}
return null;
}
if (clientContext.getDisableLegacyBlockReaderLocal()) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": can't construct BlockReaderLocalLegacy because " +
"disableLegacyBlockReaderLocal is set.");
}
return null;
}
IOException ioe = null;
try {
return BlockReaderLocalLegacy.newBlockReader(conf,
userGroupInformation, configuration, fileName, block, token,
datanode, startOffset, length);
} catch (RemoteException remoteException) {
ioe = remoteException.unwrapRemoteException(
InvalidToken.class, AccessControlException.class);
} catch (IOException e) {
ioe = e;
}
if ((!(ioe instanceof AccessControlException)) &&
isSecurityException(ioe)) {
// Handle security exceptions.
// We do not handle AccessControlException here, since
// BlockReaderLocalLegacy#newBlockReader uses that exception to indicate
// that the user is not in dfs.block.local-path-access.user, a condition
// which requires us to disable legacy SCR.
throw ioe;
}
LOG.warn(this + ": error creating legacy BlockReaderLocal. " +
"Disabling legacy local reads.", ioe);
clientContext.setDisableLegacyBlockReaderLocal();
return null;
}
private BlockReader getBlockReaderLocal() throws InvalidToken {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": trying to construct a BlockReaderLocal " +
"for short-circuit reads.");
}
if (pathInfo == null) {
pathInfo = clientContext.getDomainSocketFactory().
getPathInfo(inetSocketAddress, conf);
}
if (!pathInfo.getPathState().getUsableForShortCircuit()) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": " + pathInfo + " is not " +
"usable for short circuit; giving up on BlockReaderLocal.");
}
return null;
}
ShortCircuitCache cache = clientContext.getShortCircuitCache();
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 =
new DataOutputStream(new BufferedOutputStream(
peer.getOutputStream()));
new Sender(out).requestShortCircuitFds(block, blockToken, 1);
DataInputStream in =
new DataInputStream(peer.getInputStream());
new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
new Sender(out).requestShortCircuitFds(block, token, 1);
DataInputStream in = new DataInputStream(peer.getInputStream());
BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
PBHelper.vintPrefixed(in));
DomainSocket sock = peer.getDomainSocket();
switch (resp.getStatus()) {
case SUCCESS:
BlockReaderLocal reader = null;
byte buf[] = new byte[1];
FileInputStream fis[] = new FileInputStream[2];
sock.recvFileInputStreams(fis, buf, 0, buf.length);
ShortCircuitReplica replica = null;
try {
reader = new BlockReaderLocal.Builder(conf).
setFilename(file).
setBlock(block).
setStartOffset(startOffset).
setStreams(fis).
setDatanodeID(datanodeID).
setVerifyChecksum(verifyChecksum).
setBlockMetadataHeader(
BlockMetadataHeader.preadHeader(fis[1].getChannel())).
setFileInputStreamCache(fisCache).
setCachingStrategy(cachingStrategy).
build();
ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
replica = new ShortCircuitReplica(key, fis[0], fis[1],
clientContext.getShortCircuitCache(), Time.monotonicNow());
} catch (IOException e) {
// This indicates an error reading from disk, or a format error. Since
// it's not a socket communication problem, we return null rather than
// throwing an exception.
LOG.warn(this + ": error creating ShortCircuitReplica.", e);
return null;
} finally {
if (reader == null) {
if (replica == null) {
IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);
}
}
return reader;
return new ShortCircuitReplicaInfo(replica);
case ERROR_UNSUPPORTED:
if (!resp.hasShortCircuitAccessVersion()) {
DFSClient.LOG.warn("short-circuit read access is disabled for " +
"DataNode " + datanodeID + ". reason: " + resp.getMessage());
domSockFactory.disableShortCircuitForPath(sock.getPath());
LOG.warn("short-circuit read access is disabled for " +
"DataNode " + datanode + ". reason: " + resp.getMessage());
clientContext.getDomainSocketFactory()
.disableShortCircuitForPath(pathInfo.getPath());
} else {
DFSClient.LOG.warn("short-circuit read access for the file " +
file + " is disabled for DataNode " + datanodeID +
LOG.warn("short-circuit read access for the file " +
fileName + " is disabled for DataNode " + datanode +
". reason: " + resp.getMessage());
}
return null;
case ERROR_ACCESS_TOKEN:
String msg = "access control error while " +
"attempting to set up short-circuit access to " +
file + resp.getMessage();
DFSClient.LOG.debug(msg);
throw new InvalidBlockTokenException(msg);
fileName + resp.getMessage();
if (LOG.isDebugEnabled()) {
LOG.debug(this + ":" + msg);
}
return new ShortCircuitReplicaInfo(new InvalidToken(msg));
default:
DFSClient.LOG.warn("error while attempting to set up short-circuit " +
"access to " + file + ": " + resp.getMessage());
domSockFactory.disableShortCircuitForPath(sock.getPath());
LOG.warn(this + "unknown response code " + resp.getStatus() + " while " +
"attempting to set up short-circuit access. " + resp.getMessage());
clientContext.getDomainSocketFactory()
.disableShortCircuitForPath(pathInfo.getPath());
return null;
}
}
/**
* Get a RemoteBlockReader that communicates over a UNIX domain socket.
*
* @return The new BlockReader, or null if we failed to create the block
* reader.
*
* @throws InvalidToken If the block token was invalid.
* Potentially other security-related execptions.
*/
private BlockReader getRemoteBlockReaderFromDomain() throws IOException {
if (pathInfo == null) {
pathInfo = clientContext.getDomainSocketFactory().
getPathInfo(inetSocketAddress, conf);
}
if (!pathInfo.getPathState().getUsableForDataTransfer()) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": not trying to create a remote block reader " +
"because the UNIX domain socket at " + pathInfo +
" is not usable.");
}
return null;
}
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": trying to create a remote block reader from the " +
"UNIX domain socket at " + pathInfo.getPath());
}
while (true) {
BlockReaderPeer curPeer = nextDomainPeer();
if (curPeer == null) break;
DomainPeer peer = (DomainPeer)curPeer.peer;
BlockReader blockReader = null;
try {
blockReader = getRemoteBlockReader(peer);
return blockReader;
} catch (IOException ioe) {
IOUtils.cleanup(LOG, peer);
if (isSecurityException(ioe)) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": got security exception while constructing " +
"a remote block reader from the unix domain socket at " +
pathInfo.getPath(), ioe);
}
throw ioe;
}
if (curPeer.fromCache) {
// Handle an I/O error we got when using a cached peer. These are
// considered less serious, because the underlying socket may be stale.
if (LOG.isDebugEnabled()) {
LOG.debug("Closed potentially stale domain peer " + peer, ioe);
}
} else {
// Handle an I/O error we got when using a newly created domain peer.
// We temporarily disable the domain socket path for a few minutes in
// this case, to prevent wasting more time on it.
LOG.warn("I/O error constructing remote block reader. Disabling " +
"domain socket " + peer.getDomainSocket(), ioe);
clientContext.getDomainSocketFactory()
.disableDomainSocketPath(pathInfo.getPath());
return null;
}
} finally {
if (blockReader == null) {
IOUtils.cleanup(LOG, peer);
}
}
}
return null;
}
/**
* Get a RemoteBlockReader that communicates over a TCP socket.
*
* @return The new BlockReader. We will not return null, but instead throw
* an exception if this fails.
*
* @throws InvalidToken
* If the block token was invalid.
* InvalidEncryptionKeyException
* If the encryption key was invalid.
* Other IOException
* If there was another problem.
*/
private BlockReader getRemoteBlockReaderFromTcp() throws IOException {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": trying to create a remote block reader from a " +
"TCP socket");
}
BlockReader blockReader = null;
while (true) {
BlockReaderPeer curPeer = null;
Peer peer = null;
try {
curPeer = nextTcpPeer();
if (curPeer == null) break;
peer = curPeer.peer;
blockReader = getRemoteBlockReader(peer);
return blockReader;
} catch (IOException ioe) {
if (isSecurityException(ioe)) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": got security exception while constructing " +
"a remote block reader from " + peer, ioe);
}
throw ioe;
}
if ((curPeer != null) && curPeer.fromCache) {
// Handle an I/O error we got when using a cached peer. These are
// considered less serious, because the underlying socket may be
// stale.
if (LOG.isDebugEnabled()) {
LOG.debug("Closed potentially stale remote peer " + peer, ioe);
}
} else {
// Handle an I/O error we got when using a newly created peer.
LOG.warn("I/O error constructing remote block reader.", ioe);
throw ioe;
}
} finally {
if (blockReader == null) {
IOUtils.cleanup(LOG, peer);
}
}
}
return null;
}
private 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)
* @param s Address of the block location
@ -246,23 +777,4 @@ public static String getFileName(final InetSocketAddress s,
final String poolId, final long blockId) {
return s.toString() + ":" + poolId + ":" + blockId;
}
/**
* Get {@link BlockReaderLocalLegacy} for short circuited local reads.
* This block reader implements the path-based style of local reads
* first introduced in HDFS-2246.
*/
static BlockReader getLegacyBlockReaderLocal(DFSClient dfsClient,
String src, ExtendedBlock blk,
Token<BlockTokenIdentifier> accessToken, DatanodeInfo chosenNode,
long offsetIntoBlock) throws InvalidToken, IOException {
try {
final long length = blk.getNumBytes() - offsetIntoBlock;
return BlockReaderLocalLegacy.newBlockReader(dfsClient, src, blk,
accessToken, chosenNode, offsetIntoBlock, length);
} catch (RemoteException re) {
throw re.unwrapRemoteException(InvalidToken.class,
AccessControlException.class);
}
}
}

View File

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

View File

@ -31,7 +31,6 @@
import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.hdfs.client.ClientMmap;
import org.apache.hadoop.hdfs.client.ClientMmapManager;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -175,19 +174,21 @@ private void removeBlockLocalPathInfo(ExtendedBlock b) {
/**
* The only way this object can be instantiated.
*/
static BlockReaderLocalLegacy newBlockReader(DFSClient dfsClient,
String file, ExtendedBlock blk, Token<BlockTokenIdentifier> token,
DatanodeInfo node, long startOffset, long length)
throws IOException {
final DFSClient.Conf conf = dfsClient.getConf();
static BlockReaderLocalLegacy newBlockReader(DFSClient.Conf conf,
UserGroupInformation userGroupInformation,
Configuration configuration, String file, ExtendedBlock blk,
Token<BlockTokenIdentifier> token, DatanodeInfo node,
long startOffset, long length) throws IOException {
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
.getIpcPort());
// check the cache first
BlockLocalPathInfo pathinfo = localDatanodeInfo.getBlockLocalPathInfo(blk);
if (pathinfo == null) {
pathinfo = getBlockPathInfo(dfsClient.ugi, blk, node,
dfsClient.getConfiguration(), dfsClient.getHdfsTimeout(), token,
if (userGroupInformation == null) {
userGroupInformation = UserGroupInformation.getCurrentUser();
}
pathinfo = getBlockPathInfo(userGroupInformation, blk, node,
configuration, conf.hdfsTimeout, token,
conf.connectToDnViaHostname);
}
@ -708,8 +709,7 @@ public boolean isShortCircuit() {
}
@Override
public ClientMmap getClientMmap(EnumSet<ReadOption> opts,
ClientMmapManager mmapManager) {
public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
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_REPLICATION_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT;
import java.io.BufferedOutputStream;
import java.io.DataInputStream;
@ -108,9 +110,10 @@
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.VolumeId;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.client.ClientMmapManager;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.hdfs.net.TcpPeerServer;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
@ -191,7 +194,7 @@
*
********************************************************/
@InterfaceAudience.Private
public class DFSClient implements java.io.Closeable {
public class DFSClient implements java.io.Closeable, RemotePeerFactory {
public static final Log LOG = LogFactory.getLog(DFSClient.class);
public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
@ -212,50 +215,13 @@ public class DFSClient implements java.io.Closeable {
final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
final FileSystem.Statistics stats;
private final String authority;
final PeerCache peerCache;
private Random r = new Random();
private SocketAddress[] localInterfaceAddrs;
private DataEncryptionKey encryptionKey;
private boolean shouldUseLegacyBlockReaderLocal;
private final CachingStrategy defaultReadCachingStrategy;
private final CachingStrategy defaultWriteCachingStrategy;
private ClientMmapManager mmapManager;
private final ClientContext clientContext;
private static final ClientMmapManagerFactory MMAP_MANAGER_FACTORY =
new ClientMmapManagerFactory();
private static final class ClientMmapManagerFactory {
private ClientMmapManager mmapManager = null;
/**
* Tracks the number of users of mmapManager.
*/
private int refcnt = 0;
synchronized ClientMmapManager get(Configuration conf) {
if (refcnt++ == 0) {
mmapManager = ClientMmapManager.fromConf(conf);
} else {
String mismatches = mmapManager.verifyConfigurationMatches(conf);
if (!mismatches.isEmpty()) {
LOG.warn("The ClientMmapManager settings you specified " +
"have been ignored because another thread created the " +
"ClientMmapManager first. " + mismatches);
}
}
return mmapManager;
}
synchronized void unref(ClientMmapManager mmapManager) {
if (this.mmapManager != mmapManager) {
throw new IllegalArgumentException();
}
if (--refcnt == 0) {
IOUtils.cleanup(LOG, mmapManager);
mmapManager = null;
}
}
}
/**
* DFSClient configuration
*/
@ -300,6 +266,11 @@ public static class Conf {
final boolean domainSocketDataTraffic;
final int shortCircuitStreamsCacheSize;
final long shortCircuitStreamsCacheExpiryMs;
final int shortCircuitMmapCacheSize;
final long shortCircuitMmapCacheExpiryMs;
final long shortCircuitMmapCacheRetryTimeout;
final long shortCircuitCacheStaleThresholdMs;
public Conf(Configuration conf) {
// The hdfsTimeout is currently the same as the ipc timeout
@ -416,6 +387,18 @@ public Conf(Configuration conf) {
shortCircuitStreamsCacheExpiryMs = conf.getLong(
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT);
shortCircuitMmapCacheSize = conf.getInt(
DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE,
DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT);
shortCircuitMmapCacheExpiryMs = conf.getLong(
DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS,
DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT);
shortCircuitMmapCacheRetryTimeout = conf.getLong(
DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS,
DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS_DEFAULT);
shortCircuitCacheStaleThresholdMs = conf.getLong(
DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS,
DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT);
}
private DataChecksum.Type getChecksumType(Configuration conf) {
@ -464,11 +447,11 @@ private DataChecksum createChecksum(ChecksumOpt userOpt)
public Conf getConf() {
return dfsClientConf;
}
Configuration getConfiguration() {
return conf;
}
/**
* A map from file names to {@link DFSOutputStream} objects
* that are currently being written by this client.
@ -477,8 +460,6 @@ Configuration getConfiguration() {
private final Map<String, DFSOutputStream> filesBeingWritten
= new HashMap<String, DFSOutputStream>();
private final DomainSocketFactory domainSocketFactory;
/**
* Same as this(NameNode.getAddress(conf), conf);
* @see #DFSClient(InetSocketAddress, Configuration)
@ -526,8 +507,6 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
throws IOException {
// Copy only the required DFSClient configuration
this.dfsClientConf = new Conf(conf);
this.shouldUseLegacyBlockReaderLocal =
this.dfsClientConf.useLegacyBlockReaderLocal;
if (this.dfsClientConf.useLegacyBlockReaderLocal) {
LOG.debug("Using legacy short-circuit local reads.");
}
@ -572,9 +551,6 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
this.namenode = proxyInfo.getProxy();
}
// read directly from the block file if configured.
this.domainSocketFactory = new DomainSocketFactory(dfsClientConf);
String localInterfaces[] =
conf.getTrimmedStrings(DFSConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces);
@ -584,7 +560,6 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
Joiner.on(',').join(localInterfaceAddrs) + "]");
}
this.peerCache = PeerCache.getInstance(dfsClientConf.socketCacheCapacity, dfsClientConf.socketCacheExpiry);
Boolean readDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_READS) == null) ?
null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_READS, false);
Long readahead = (conf.get(DFS_CLIENT_CACHE_READAHEAD) == null) ?
@ -595,7 +570,9 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
new CachingStrategy(readDropBehind, readahead);
this.defaultWriteCachingStrategy =
new CachingStrategy(writeDropBehind, readahead);
this.mmapManager = MMAP_MANAGER_FACTORY.get(conf);
this.clientContext = ClientContext.get(
conf.get(DFS_CLIENT_CONTEXT, DFS_CLIENT_CONTEXT_DEFAULT),
dfsClientConf);
}
/**
@ -800,10 +777,6 @@ void closeConnectionToNamenode() {
/** Abort and release resources held. Ignore all errors. */
void abort() {
if (mmapManager != null) {
MMAP_MANAGER_FACTORY.unref(mmapManager);
mmapManager = null;
}
clientRunning = false;
closeAllFilesBeingWritten(true);
try {
@ -849,10 +822,6 @@ private void closeAllFilesBeingWritten(final boolean abort) {
*/
@Override
public synchronized void close() throws IOException {
if (mmapManager != null) {
MMAP_MANAGER_FACTORY.unref(mmapManager);
mmapManager = null;
}
if(clientRunning) {
closeAllFilesBeingWritten(false);
clientRunning = false;
@ -2626,18 +2595,6 @@ public String toString() {
+ ", ugi=" + ugi + "]";
}
public DomainSocketFactory getDomainSocketFactory() {
return domainSocketFactory;
}
public void disableLegacyBlockReaderLocal() {
shouldUseLegacyBlockReaderLocal = false;
}
public boolean useLegacyBlockReaderLocal() {
return shouldUseLegacyBlockReaderLocal;
}
public CachingStrategy getDefaultReadCachingStrategy() {
return defaultReadCachingStrategy;
}
@ -2646,8 +2603,29 @@ public CachingStrategy getDefaultWriteCachingStrategy() {
return defaultWriteCachingStrategy;
}
@VisibleForTesting
public ClientMmapManager getMmapManager() {
return mmapManager;
public ClientContext getClientContext() {
return clientContext;
}
@Override // RemotePeerFactory
public Peer newConnectedPeer(InetSocketAddress addr) throws IOException {
Peer peer = null;
boolean success = false;
Socket sock = null;
try {
sock = socketFactory.createSocket();
NetUtils.connect(sock, addr,
getRandomLocalInterfaceAddr(),
dfsClientConf.socketTimeout);
peer = TcpPeerServer.peerFromSocketAndKey(sock,
getDataEncryptionKey());
success = true;
return peer;
} finally {
if (!success) {
IOUtils.cleanup(LOG, peer);
IOUtils.closeSocket(sock);
}
}
}
}

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_READS = "dfs.client.cache.drop.behind.reads";
public static final String DFS_CLIENT_CACHE_READAHEAD = "dfs.client.cache.readahead";
public static final String DFS_CLIENT_CONTEXT = "dfs.client.context";
public static final String DFS_CLIENT_CONTEXT_DEFAULT = "default";
public static final String DFS_HDFS_BLOCKS_METADATA_ENABLED = "dfs.datanode.hdfs-blocks-metadata.enabled";
public static final boolean DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT = false;
public static final String DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS = "dfs.client.file-block-storage-locations.num-threads";
@ -418,18 +420,20 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final boolean DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT = false;
public static final String DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY = "dfs.client.read.shortcircuit.buffer.size";
public static final String DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY = "dfs.client.read.shortcircuit.streams.cache.size";
public static final int DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT = 100;
public static final int DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT = 256;
public static final String DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY = "dfs.client.read.shortcircuit.streams.cache.expiry.ms";
public static final long DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT = 5000;
public static final long DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT = 5 * 60 * 1000;
public static final int DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT = 1024 * 1024;
public static final String DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC = "dfs.client.domain.socket.data.traffic";
public static final boolean DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT = false;
public static final String DFS_CLIENT_MMAP_CACHE_SIZE = "dfs.client.mmap.cache.size";
public static final int DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT = 1024;
public static final int DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT = 256;
public static final String DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS = "dfs.client.mmap.cache.timeout.ms";
public static final long DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT = 15 * 60 * 1000;
public static final String DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT = "dfs.client.mmap.cache.thread.runs.per.timeout";
public static final int DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT_DEFAULT = 4;
public static final long DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT = 60 * 60 * 1000;
public static final String DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS = "dfs.client.mmap.retry.timeout.ms";
public static final long DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS_DEFAULT = 5 * 60 * 1000;
public static final String DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS = "dfs.client.short.circuit.replica.stale.threshold.ms";
public static final long DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT = 30 * 60 * 1000;
// property for fsimage compression
public static final String DFS_IMAGE_COMPRESS_KEY = "dfs.image.compress";

View File

@ -46,9 +46,6 @@
import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.hdfs.client.ClientMmap;
import org.apache.hadoop.hdfs.net.DomainPeer;
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.hdfs.net.TcpPeerServer;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -82,7 +79,6 @@ public class DFSInputStream extends FSInputStream
HasEnhancedByteBufferAccess {
@VisibleForTesting
static boolean tcpReadsDisabledForTesting = false;
private final PeerCache peerCache;
private final DFSClient dfsClient;
private boolean closed = false;
private final String src;
@ -190,8 +186,6 @@ void addZeroCopyBytes(long amt) {
private long totalZeroCopyBytesRead;
}
private final FileInputStreamCache fileInputStreamCache;
/**
* This variable tracks the number of failures since the start of the
* most recent user-facing operation. That is to say, it should be reset
@ -223,10 +217,6 @@ void addToDeadNodes(DatanodeInfo dnInfo) {
this.verifyChecksum = verifyChecksum;
this.buffersize = buffersize;
this.src = src;
this.peerCache = dfsClient.peerCache;
this.fileInputStreamCache = new FileInputStreamCache(
dfsClient.getConf().shortCircuitStreamsCacheSize,
dfsClient.getConf().shortCircuitStreamsCacheExpiryMs);
this.cachingStrategy =
dfsClient.getDefaultReadCachingStrategy();
openInfo();
@ -572,18 +562,28 @@ private synchronized DatanodeInfo blockSeekTo(long target) throws IOException {
try {
ExtendedBlock blk = targetBlock.getBlock();
Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
blockReader = getBlockReader(targetAddr, chosenNode, src, blk,
accessToken, offsetIntoBlock, blk.getNumBytes() - offsetIntoBlock,
buffersize, verifyChecksum, dfsClient.clientName, cachingStrategy);
blockReader = new BlockReaderFactory(dfsClient.getConf()).
setInetSocketAddress(targetAddr).
setRemotePeerFactory(dfsClient).
setDatanodeInfo(chosenNode).
setFileName(src).
setBlock(blk).
setBlockToken(accessToken).
setStartOffset(offsetIntoBlock).
setVerifyChecksum(verifyChecksum).
setClientName(dfsClient.clientName).
setLength(blk.getNumBytes() - offsetIntoBlock).
setCachingStrategy(cachingStrategy).
setAllowShortCircuitLocalReads(!shortCircuitForbidden()).
setClientCacheContext(dfsClient.getClientContext()).
setUserGroupInformation(dfsClient.ugi).
setConfiguration(dfsClient.getConfiguration()).
build();
if(connectFailedOnce) {
DFSClient.LOG.info("Successfully connected to " + targetAddr +
" for " + blk);
}
return chosenNode;
} catch (AccessControlException ex) {
DFSClient.LOG.warn("Short circuit access failed " + ex);
dfsClient.disableLegacyBlockReaderLocal();
continue;
} catch (IOException ex) {
if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
DFSClient.LOG.info("Will fetch a new encryption key and retry, "
@ -635,7 +635,6 @@ public void accept(ByteBuffer k, Object v) {
blockReader = null;
}
super.close();
fileInputStreamCache.close();
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
// start of the loop.
CachingStrategy curCachingStrategy;
boolean allowShortCircuitLocalReads;
synchronized (this) {
block = getBlockAt(block.getStartOffset(), false);
curCachingStrategy = cachingStrategy;
allowShortCircuitLocalReads = !shortCircuitForbidden();
}
DNAddrPair retval = chooseDataNode(block);
DatanodeInfo chosenNode = retval.info;
@ -943,11 +944,24 @@ private void fetchBlockByteRange(LocatedBlock block, long start, long end,
try {
Token<BlockTokenIdentifier> blockToken = block.getBlockToken();
int len = (int) (end - start + 1);
reader = getBlockReader(targetAddr, chosenNode, src, block.getBlock(),
blockToken, start, len, buffersize, verifyChecksum,
dfsClient.clientName, curCachingStrategy);
reader = new BlockReaderFactory(dfsClient.getConf()).
setInetSocketAddress(targetAddr).
setRemotePeerFactory(dfsClient).
setDatanodeInfo(chosenNode).
setFileName(src).
setBlock(block.getBlock()).
setBlockToken(blockToken).
setStartOffset(start).
setVerifyChecksum(verifyChecksum).
setClientName(dfsClient.clientName).
setLength(len).
setCachingStrategy(curCachingStrategy).
setAllowShortCircuitLocalReads(allowShortCircuitLocalReads).
setClientCacheContext(dfsClient.getClientContext()).
setUserGroupInformation(dfsClient.ugi).
setConfiguration(dfsClient.getConfiguration()).
build();
int nread = reader.readAll(buf, offset, len);
if (nread != len) {
throw new IOException("truncated return from reader.read(): " +
@ -960,10 +974,6 @@ private void fetchBlockByteRange(LocatedBlock block, long start, long end,
e.getPos() + " from " + chosenNode);
// we want to remember what we have tried
addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
} catch (AccessControlException ex) {
DFSClient.LOG.warn("Short circuit access failed " + ex);
dfsClient.disableLegacyBlockReaderLocal();
continue;
} catch (IOException e) {
if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
DFSClient.LOG.info("Will fetch a new encryption key and retry, "
@ -1022,194 +1032,6 @@ private static boolean tokenRefetchNeeded(IOException ex,
return false;
}
private Peer newTcpPeer(InetSocketAddress addr) throws IOException {
Peer peer = null;
boolean success = false;
Socket sock = null;
try {
sock = dfsClient.socketFactory.createSocket();
NetUtils.connect(sock, addr,
dfsClient.getRandomLocalInterfaceAddr(),
dfsClient.getConf().socketTimeout);
peer = TcpPeerServer.peerFromSocketAndKey(sock,
dfsClient.getDataEncryptionKey());
success = true;
return peer;
} finally {
if (!success) {
IOUtils.closeQuietly(peer);
IOUtils.closeQuietly(sock);
}
}
}
/**
* Retrieve a BlockReader suitable for reading.
* This method will reuse the cached connection to the DN if appropriate.
* Otherwise, it will create a new connection.
* Throwing an IOException from this method is basically equivalent to
* declaring the DataNode bad, so we try to connect a lot of different ways
* before doing that.
*
* @param dnAddr Address of the datanode
* @param chosenNode Chosen datanode information
* @param file File location
* @param block The Block object
* @param blockToken The access token for security
* @param startOffset The read offset, relative to block head
* @param len The number of bytes to read
* @param bufferSize The IO buffer size (not the client buffer size)
* @param verifyChecksum Whether to verify checksum
* @param clientName Client name
* @param CachingStrategy caching strategy to use
* @return New BlockReader instance
*/
protected BlockReader getBlockReader(InetSocketAddress dnAddr,
DatanodeInfo chosenNode,
String file,
ExtendedBlock block,
Token<BlockTokenIdentifier> blockToken,
long startOffset,
long len,
int bufferSize,
boolean verifyChecksum,
String clientName,
CachingStrategy curCachingStrategy)
throws IOException {
// Firstly, we check to see if we have cached any file descriptors for
// local blocks. If so, we can just re-use those file descriptors.
FileInputStream fis[] = fileInputStreamCache.get(chosenNode, block);
if (fis != null) {
if (DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("got FileInputStreams for " + block + " from " +
"the FileInputStreamCache.");
}
return new BlockReaderLocal.Builder(dfsClient.getConf()).
setFilename(file).
setBlock(block).
setStartOffset(startOffset).
setStreams(fis).
setDatanodeID(chosenNode).
setVerifyChecksum(verifyChecksum).
setBlockMetadataHeader(BlockMetadataHeader.
preadHeader(fis[1].getChannel())).
setFileInputStreamCache(fileInputStreamCache).
setCachingStrategy(curCachingStrategy).
build();
}
// If the legacy local block reader is enabled and we are reading a local
// block, try to create a BlockReaderLocalLegacy. The legacy local block
// reader implements local reads in the style first introduced by HDFS-2246.
if ((dfsClient.useLegacyBlockReaderLocal()) &&
DFSClient.isLocalAddress(dnAddr) &&
(!shortCircuitForbidden())) {
try {
return BlockReaderFactory.getLegacyBlockReaderLocal(dfsClient,
clientName, block, blockToken, chosenNode, startOffset);
} catch (IOException e) {
DFSClient.LOG.warn("error creating legacy BlockReaderLocal. " +
"Disabling legacy local reads.", e);
dfsClient.disableLegacyBlockReaderLocal();
}
}
// Look for cached domain peers.
int cacheTries = 0;
DomainSocketFactory dsFactory = dfsClient.getDomainSocketFactory();
BlockReader reader = null;
final int nCachedConnRetry = dfsClient.getConf().nCachedConnRetry;
for (; cacheTries < nCachedConnRetry; ++cacheTries) {
Peer peer = peerCache.get(chosenNode, true);
if (peer == null) break;
try {
boolean allowShortCircuitLocalReads = dfsClient.getConf().
shortCircuitLocalReads && (!shortCircuitForbidden());
reader = BlockReaderFactory.newBlockReader(
dfsClient.getConf(), file, block, blockToken, startOffset,
len, verifyChecksum, clientName, peer, chosenNode,
dsFactory, peerCache, fileInputStreamCache,
allowShortCircuitLocalReads, curCachingStrategy);
return reader;
} catch (IOException ex) {
DFSClient.LOG.debug("Error making BlockReader with DomainSocket. " +
"Closing stale " + peer, ex);
} finally {
if (reader == null) {
IOUtils.closeQuietly(peer);
}
}
}
// Try to create a DomainPeer.
DomainSocket domSock = dsFactory.create(dnAddr, this);
if (domSock != null) {
Peer peer = new DomainPeer(domSock);
try {
boolean allowShortCircuitLocalReads = dfsClient.getConf().
shortCircuitLocalReads && (!shortCircuitForbidden());
reader = BlockReaderFactory.newBlockReader(
dfsClient.getConf(), file, block, blockToken, startOffset,
len, verifyChecksum, clientName, peer, chosenNode,
dsFactory, peerCache, fileInputStreamCache,
allowShortCircuitLocalReads, curCachingStrategy);
return reader;
} catch (IOException e) {
DFSClient.LOG.warn("failed to connect to " + domSock, e);
} finally {
if (reader == null) {
// If the Peer that we got the error from was a DomainPeer,
// mark the socket path as bad, so that newDataSocket will not try
// to re-open this socket for a while.
dsFactory.disableDomainSocketPath(domSock.getPath());
IOUtils.closeQuietly(peer);
}
}
}
// Look for cached peers.
for (; cacheTries < nCachedConnRetry; ++cacheTries) {
Peer peer = peerCache.get(chosenNode, false);
if (peer == null) break;
try {
reader = BlockReaderFactory.newBlockReader(
dfsClient.getConf(), file, block, blockToken, startOffset,
len, verifyChecksum, clientName, peer, chosenNode,
dsFactory, peerCache, fileInputStreamCache, false,
curCachingStrategy);
return reader;
} catch (IOException ex) {
DFSClient.LOG.debug("Error making BlockReader. Closing stale " +
peer, ex);
} finally {
if (reader == null) {
IOUtils.closeQuietly(peer);
}
}
}
if (tcpReadsDisabledForTesting) {
throw new IOException("TCP reads are disabled.");
}
// Try to create a new remote peer.
Peer peer = newTcpPeer(dnAddr);
try {
reader = BlockReaderFactory.newBlockReader(dfsClient.getConf(), file,
block, blockToken, startOffset, len, verifyChecksum, clientName,
peer, chosenNode, dsFactory, peerCache, fileInputStreamCache, false,
curCachingStrategy);
return reader;
} catch (IOException ex) {
DFSClient.LOG.debug(
"Exception while getting block reader, closing stale " + peer, ex);
throw ex;
} finally {
if (reader == null) {
IOUtils.closeQuietly(peer);
}
}
}
/**
* Read bytes starting from the specified position.
*
@ -1555,8 +1377,7 @@ private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
long blockStartInFile = currentLocatedBlock.getStartOffset();
long blockPos = curPos - blockStartInFile;
long limit = blockPos + length;
ClientMmap clientMmap =
blockReader.getClientMmap(opts, dfsClient.getMmapManager());
ClientMmap clientMmap = blockReader.getClientMmap(opts);
if (clientMmap == null) {
if (DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
@ -1565,17 +1386,25 @@ private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
}
return null;
}
seek(pos + length);
ByteBuffer buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer();
buffer.position((int)blockPos);
buffer.limit((int)limit);
clientMmap.ref();
extendedReadBuffers.put(buffer, clientMmap);
readStatistics.addZeroCopyBytes(length);
if (DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("readZeroCopy read " + maxLength + " bytes from " +
"offset " + curPos + " via the zero-copy read path. " +
"blockEnd = " + blockEnd);
boolean success = false;
ByteBuffer buffer;
try {
seek(pos + length);
buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer();
buffer.position((int)blockPos);
buffer.limit((int)limit);
extendedReadBuffers.put(buffer, clientMmap);
readStatistics.addZeroCopyBytes(length);
if (DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("readZeroCopy read " + maxLength + " bytes from " +
"offset " + curPos + " via the zero-copy read path. " +
"blockEnd = " + blockEnd);
}
success = true;
} finally {
if (!success) {
clientMmap.unref();
}
}
return buffer;
}

View File

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

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();
private final int capacity;
private final long expiryPeriod;
private static PeerCache instance = null;
@VisibleForTesting
PeerCache(int c, long e) {
public PeerCache(int c, long e) {
this.capacity = c;
this.expiryPeriod = e;
if (capacity == 0 ) {
LOG.info("SocketCache disabled.");
}
else if (expiryPeriod == 0) {
} else if (expiryPeriod == 0) {
throw new IllegalStateException("Cannot initialize expiryPeriod to " +
expiryPeriod + "when cache is enabled.");
expiryPeriod + " when cache is enabled.");
}
}
public static synchronized PeerCache getInstance(int c, long e) {
// capacity is only initialized once
if (instance == null) {
instance = new PeerCache(c, e);
} else { //already initialized once
if (instance.capacity != c || instance.expiryPeriod != e) {
LOG.info("capacity and expiry periods already set to " +
instance.capacity + " and " + instance.expiryPeriod +
" respectively. Cannot set it to " + c + " and " + e);
}
}
return instance;
}
@VisibleForTesting
public static synchronized void setInstance(int c, long e) {
instance = new PeerCache(c, e);
}
private boolean isDaemonStarted() {
return (daemon == null)? false: true;
}

View File

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

View File

@ -32,7 +32,6 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.hdfs.client.ClientMmap;
import org.apache.hadoop.hdfs.client.ClientMmapManager;
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -457,8 +456,7 @@ public boolean isShortCircuit() {
}
@Override
public ClientMmap getClientMmap(EnumSet<ReadOption> opts,
ClientMmapManager mmapManager) {
public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
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;
import java.io.FileInputStream;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.io.nativeio.NativeIO;
import java.io.IOException;
import java.lang.ref.WeakReference;
import java.nio.MappedByteBuffer;
import java.nio.channels.FileChannel.MapMode;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import com.google.common.annotations.VisibleForTesting;
/**
* A memory-mapped region used by an HDFS client.
*
@ -46,111 +36,46 @@ public class ClientMmap {
static final Log LOG = LogFactory.getLog(ClientMmap.class);
/**
* A reference to the manager of this mmap.
*
* This is only a weak reference to help minimize the damange done by
* code which leaks references accidentally.
* A reference to the block replica which this mmap relates to.
*/
private final WeakReference<ClientMmapManager> manager;
private final ShortCircuitReplica replica;
/**
* The actual mapped memory region.
* The java ByteBuffer object.
*/
private final MappedByteBuffer map;
/**
* A reference count tracking how many threads are using this object.
* Reference count of this ClientMmap object.
*/
private final AtomicInteger refCount = new AtomicInteger(1);
/**
* Block pertaining to this mmap
*/
private final ExtendedBlock block;
/**
* The DataNode where this mmap came from.
*/
private final DatanodeID datanodeID;
/**
* The monotonic time when this mmap was last evictable.
*/
private long lastEvictableTimeNs;
public static ClientMmap load(ClientMmapManager manager, FileInputStream in,
ExtendedBlock block, DatanodeID datanodeID)
throws IOException {
MappedByteBuffer map =
in.getChannel().map(MapMode.READ_ONLY, 0,
in.getChannel().size());
return new ClientMmap(manager, map, block, datanodeID);
}
private ClientMmap(ClientMmapManager manager, MappedByteBuffer map,
ExtendedBlock block, DatanodeID datanodeID)
throws IOException {
this.manager = new WeakReference<ClientMmapManager>(manager);
ClientMmap(ShortCircuitReplica replica, MappedByteBuffer map) {
this.replica = replica;
this.map = map;
this.block = block;
this.datanodeID = datanodeID;
this.lastEvictableTimeNs = 0;
}
/**
* Decrement the reference count on this object.
* Should be called with the ClientMmapManager lock held.
* Increment the reference count.
*
* @return The new reference count.
*/
void ref() {
refCount.addAndGet(1);
}
/**
* Decrement the reference count.
*
* The parent replica gets unreferenced each time the reference count
* of this object goes to 0.
*/
public void unref() {
int count = refCount.decrementAndGet();
if (count < 0) {
throw new IllegalArgumentException("can't decrement the " +
"reference count on this ClientMmap lower than 0.");
} else if (count == 0) {
ClientMmapManager man = manager.get();
if (man == null) {
unmap();
} else {
man.makeEvictable(this);
}
}
}
/**
* Increment the reference count on this object.
*
* @return The new reference count.
*/
public int ref() {
return refCount.getAndIncrement();
}
@VisibleForTesting
public ExtendedBlock getBlock() {
return block;
}
DatanodeID getDatanodeID() {
return datanodeID;
refCount.addAndGet(-1);
replica.unref();
}
public MappedByteBuffer getMappedByteBuffer() {
return map;
}
public void setLastEvictableTimeNs(long lastEvictableTimeNs) {
this.lastEvictableTimeNs = lastEvictableTimeNs;
}
public long getLastEvictableTimeNs() {
return this.lastEvictableTimeNs;
}
/**
* Unmap the memory region.
*/
void unmap() {
assert(refCount.get() == 0);
NativeIO.POSIX.munmap(map);
}
}
}

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

@ -121,7 +121,8 @@ public static enum Feature implements LayoutFeature {
ADD_DATANODE_AND_STORAGE_UUIDS(-49, "Replace StorageID with DatanodeUuid."
+ " Use distinct StorageUuid per storage directory."),
ADD_LAYOUT_FLAGS(-50, "Add support for layout flags."),
CACHING(-51, "Support for cache pools and path-based caching");
CACHING(-51, "Support for cache pools and path-based caching"),
PROTOBUF_FORMAT(-52, "Use protobuf to serialize FSImage");
private final FeatureInfo info;

View File

@ -103,9 +103,10 @@ public DatanodeProtocolClientSideTranslatorPB(InetSocketAddress nameNodeAddr,
private static DatanodeProtocolPB createNamenode(
InetSocketAddress nameNodeAddr, Configuration conf,
UserGroupInformation ugi) throws IOException {
return RPC.getProxy(DatanodeProtocolPB.class,
return RPC.getProtocolProxy(DatanodeProtocolPB.class,
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 */

View File

@ -23,12 +23,16 @@
import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map.Entry;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
@ -46,6 +50,10 @@
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
import org.apache.hadoop.security.token.delegation.DelegationKey;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.protobuf.ByteString;
/**
* A HDFS specific delegation token secret manager.
* The secret manager is responsible for generating and accepting the password
@ -167,7 +175,45 @@ public synchronized void loadSecretManagerStateCompat(DataInput in)
}
serializerCompat.load(in);
}
public static class SecretManagerState {
public final SecretManagerSection section;
public final List<SecretManagerSection.DelegationKey> keys;
public final List<SecretManagerSection.PersistToken> tokens;
public SecretManagerState(
SecretManagerSection s,
List<SecretManagerSection.DelegationKey> keys,
List<SecretManagerSection.PersistToken> tokens) {
this.section = s;
this.keys = keys;
this.tokens = tokens;
}
}
public synchronized void loadSecretManagerState(SecretManagerState state)
throws IOException {
Preconditions.checkState(!running,
"Can't load state from image in a running SecretManager.");
currentId = state.section.getCurrentId();
delegationTokenSequenceNumber = state.section.getTokenSequenceNumber();
for (SecretManagerSection.DelegationKey k : state.keys) {
addKey(new DelegationKey(k.getId(), k.getExpiryDate(), k.hasKey() ? k
.getKey().toByteArray() : null));
}
for (SecretManagerSection.PersistToken t : state.tokens) {
DelegationTokenIdentifier id = new DelegationTokenIdentifier(new Text(
t.getOwner()), new Text(t.getRenewer()), new Text(t.getRealUser()));
id.setIssueDate(t.getIssueDate());
id.setMaxDate(t.getMaxDate());
id.setSequenceNumber(t.getSequenceNumber());
id.setMasterKeyId(t.getMasterKeyId());
addPersistedDelegationToken(id, t.getExpiryDate());
}
}
/**
* Store the current state of the SecretManager for persistence
*
@ -179,7 +225,43 @@ public synchronized void saveSecretManagerStateCompat(DataOutputStream out,
String sdPath) throws IOException {
serializerCompat.save(out, sdPath);
}
public synchronized SecretManagerState saveSecretManagerState() {
SecretManagerSection s = SecretManagerSection.newBuilder()
.setCurrentId(currentId)
.setTokenSequenceNumber(delegationTokenSequenceNumber)
.setNumKeys(allKeys.size()).setNumTokens(currentTokens.size()).build();
ArrayList<SecretManagerSection.DelegationKey> keys = Lists
.newArrayListWithCapacity(allKeys.size());
ArrayList<SecretManagerSection.PersistToken> tokens = Lists
.newArrayListWithCapacity(currentTokens.size());
for (DelegationKey v : allKeys.values()) {
SecretManagerSection.DelegationKey.Builder b = SecretManagerSection.DelegationKey
.newBuilder().setId(v.getKeyId()).setExpiryDate(v.getExpiryDate());
if (v.getEncodedKey() != null) {
b.setKey(ByteString.copyFrom(v.getEncodedKey()));
}
keys.add(b.build());
}
for (Entry<DelegationTokenIdentifier, DelegationTokenInformation> e : currentTokens
.entrySet()) {
DelegationTokenIdentifier id = e.getKey();
SecretManagerSection.PersistToken.Builder b = SecretManagerSection.PersistToken
.newBuilder().setOwner(id.getOwner().toString())
.setRenewer(id.getRenewer().toString())
.setRealUser(id.getRealUser().toString())
.setIssueDate(id.getIssueDate()).setMaxDate(id.getMaxDate())
.setSequenceNumber(id.getSequenceNumber())
.setMasterKeyId(id.getMasterKeyId())
.setExpiryDate(e.getValue().getRenewDate());
tokens.add(b.build());
}
return new SecretManagerState(s, keys, tokens);
}
/**
* This method is intended to be used only while reading edit logs.
*
@ -431,4 +513,5 @@ private synchronized void loadAllKeys(DataInput in) throws IOException {
prog.endStep(Phase.LOADING_FSIMAGE, step);
}
}
}

View File

@ -633,9 +633,11 @@ private boolean isGoodTarget(DatanodeStorageInfo storage,
// check the communication traffic of the target machine
if (considerLoad) {
double avgLoad = 0;
int size = clusterMap.getNumOfLeaves();
if (size != 0 && stats != null) {
avgLoad = (double)stats.getTotalLoad()/size;
if (stats != null) {
int size = stats.getNumDatanodesInService();
if (size != 0) {
avgLoad = (double)stats.getTotalLoad()/size;
}
}
if (node.getXceiverCount() > (2.0 * avgLoad)) {
logNodeIsNotChosen(storage, "the node is too busy ");

View File

@ -18,7 +18,26 @@
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.LogFactory;
@ -27,10 +46,17 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.BlockReader;
import org.apache.hadoop.hdfs.BlockReaderFactory;
import org.apache.hadoop.hdfs.ClientContext;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.RemotePeerFactory;
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.hdfs.net.TcpPeerServer;
import org.apache.hadoop.hdfs.protocol.*;
import org.apache.hadoop.hdfs.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.DataEncryptionKey;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@ -53,22 +79,7 @@
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.VersionInfo;
import javax.servlet.ServletContext;
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;
import com.google.common.base.Charsets;
@InterfaceAudience.Private
public class JspHelper {
@ -168,101 +179,97 @@ public static DatanodeInfo bestNode(LocatedBlocks blks, Configuration conf)
}
NodeRecord[] nodes = map.values().toArray(new NodeRecord[map.size()]);
Arrays.sort(nodes, new NodeRecordComparator());
return bestNode(nodes, false, conf);
return bestNode(nodes, false);
}
public static DatanodeInfo bestNode(LocatedBlock blk, Configuration conf)
throws IOException {
DatanodeInfo[] nodes = blk.getLocations();
return bestNode(nodes, true, conf);
return bestNode(nodes, true);
}
public static DatanodeInfo bestNode(DatanodeInfo[] nodes, boolean doRandom,
Configuration conf) throws IOException {
TreeSet<DatanodeInfo> deadNodes = new TreeSet<DatanodeInfo>();
DatanodeInfo chosenNode = null;
int failures = 0;
Socket s = null;
int index = -1;
private static DatanodeInfo bestNode(DatanodeInfo[] nodes, boolean doRandom)
throws IOException {
if (nodes == null || nodes.length == 0) {
throw new IOException("No nodes contain this block");
}
while (s == null) {
if (chosenNode == null) {
do {
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
InetSocketAddress targetAddr = NetUtils.createSocketAddr(
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 l = 0;
while (l < nodes.length && !nodes[l].isDecommissioned()) {
++l;
}
s.close();
return chosenNode;
if (l == 0) {
throw new IOException("No active nodes contain this block");
}
int index = doRandom ? DFSUtil.getRandom().nextInt(l) : 0;
return nodes[index];
}
public static void streamBlockInAscii(InetSocketAddress addr, String poolId,
long blockId, Token<BlockTokenIdentifier> blockToken, long genStamp,
long blockSize, long offsetIntoBlock, long chunkSizeToView,
JspWriter out, Configuration conf, DFSClient.Conf dfsConf,
DataEncryptionKey encryptionKey)
JspWriter out, final Configuration conf, DFSClient.Conf dfsConf,
final DataEncryptionKey encryptionKey)
throws IOException {
if (chunkSizeToView == 0) return;
Socket s = NetUtils.getDefaultSocketFactory(conf).createSocket();
s.connect(addr, HdfsServerConstants.READ_TIMEOUT);
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
int amtToRead = (int)Math.min(chunkSizeToView, blockSize - offsetIntoBlock);
// Use the block name for file name.
String file = BlockReaderFactory.getFileName(addr, poolId, blockId);
BlockReader blockReader = BlockReaderFactory.newBlockReader(dfsConf, file,
new ExtendedBlock(poolId, blockId, 0, genStamp), blockToken,
offsetIntoBlock, amtToRead, true,
"JspHelper", TcpPeerServer.peerFromSocketAndKey(s, encryptionKey),
new DatanodeID(addr.getAddress().getHostAddress(),
addr.getHostName(), poolId, addr.getPort(), 0, 0, 0), null,
null, null, false, CachingStrategy.newDefaultStrategy());
BlockReader blockReader = new BlockReaderFactory(dfsConf).
setInetSocketAddress(addr).
setBlock(new ExtendedBlock(poolId, blockId, 0, genStamp)).
setFileName(BlockReaderFactory.getFileName(addr, poolId, blockId)).
setBlockToken(blockToken).
setStartOffset(offsetIntoBlock).
setLength(amtToRead).
setVerifyChecksum(true).
setClientName("JspHelper").
setClientCacheContext(ClientContext.getFromConf(conf)).
setDatanodeInfo(new DatanodeInfo(
new DatanodeID(addr.getAddress().getHostAddress(),
addr.getHostName(), poolId, addr.getPort(), 0, 0, 0))).
setCachingStrategy(CachingStrategy.newDefaultStrategy()).
setConfiguration(conf).
setRemotePeerFactory(new RemotePeerFactory() {
@Override
public Peer newConnectedPeer(InetSocketAddress addr)
throws IOException {
Peer peer = null;
Socket sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
try {
sock.connect(addr, HdfsServerConstants.READ_TIMEOUT);
sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
peer = TcpPeerServer.peerFromSocketAndKey(sock, encryptionKey);
} finally {
if (peer == null) {
IOUtils.closeSocket(sock);
}
}
return peer;
}
}).
build();
final byte[] buf = new byte[amtToRead];
int readOffset = 0;
int retries = 2;
while ( amtToRead > 0 ) {
int numRead = amtToRead;
try {
blockReader.readFully(buf, readOffset, amtToRead);
try {
int readOffset = 0;
int retries = 2;
while (amtToRead > 0) {
int numRead = amtToRead;
try {
blockReader.readFully(buf, readOffset, amtToRead);
} catch (IOException e) {
retries--;
if (retries == 0)
throw new IOException("Could not read data from datanode");
continue;
}
amtToRead -= numRead;
readOffset += numRead;
}
catch (IOException e) {
retries--;
if (retries == 0)
throw new IOException("Could not read data from datanode");
continue;
}
amtToRead -= numRead;
readOffset += numRead;
} finally {
blockReader.close();
}
blockReader.close();
out.print(HtmlQuoting.quoteHtmlChars(new String(buf, Charsets.UTF_8)));
}

View File

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

View File

@ -2525,7 +2525,7 @@ public void clearAllBlockSecretKeys() {
/**
* Get current value of the max balancer bandwidth in bytes per second.
*
* @return bandwidth Blanacer bandwidth in bytes per second for this datanode.
* @return Balancer bandwidth in bytes per second for this datanode.
*/
public Long getBalancerBandwidth() {
DataXceiverServer dxcs =

View File

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

View File

@ -50,8 +50,10 @@
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
import org.apache.hadoop.fs.CacheFlag;
import org.apache.hadoop.fs.InvalidRequestException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.CacheDirective;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
@ -62,11 +64,15 @@
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.CacheManagerSection;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
@ -81,6 +87,7 @@
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Lists;
/**
* The Cache Manager handles caching on DataNodes.
@ -167,6 +174,19 @@ public final class CacheManager {
*/
private CacheReplicationMonitor monitor;
public static final class PersistState {
public final CacheManagerSection section;
public final List<CachePoolInfoProto> pools;
public final List<CacheDirectiveInfoProto> directives;
public PersistState(CacheManagerSection section,
List<CachePoolInfoProto> pools, List<CacheDirectiveInfoProto> directives) {
this.section = section;
this.pools = pools;
this.directives = directives;
}
}
CacheManager(FSNamesystem namesystem, Configuration conf,
BlockManager blockManager) {
this.namesystem = namesystem;
@ -944,6 +964,64 @@ public void saveStateCompat(DataOutputStream out, String sdPath)
serializerCompat.save(out, sdPath);
}
public PersistState saveState() throws IOException {
ArrayList<CachePoolInfoProto> pools = Lists
.newArrayListWithCapacity(cachePools.size());
ArrayList<CacheDirectiveInfoProto> directives = Lists
.newArrayListWithCapacity(directivesById.size());
for (CachePool pool : cachePools.values()) {
CachePoolInfo p = pool.getInfo(true);
CachePoolInfoProto.Builder b = CachePoolInfoProto.newBuilder()
.setPoolName(p.getPoolName());
if (p.getOwnerName() != null)
b.setOwnerName(p.getOwnerName());
if (p.getGroupName() != null)
b.setGroupName(p.getGroupName());
if (p.getMode() != null)
b.setMode(p.getMode().toShort());
if (p.getLimit() != null)
b.setLimit(p.getLimit());
pools.add(b.build());
}
for (CacheDirective directive : directivesById.values()) {
CacheDirectiveInfo info = directive.toInfo();
CacheDirectiveInfoProto.Builder b = CacheDirectiveInfoProto.newBuilder()
.setId(info.getId());
if (info.getPath() != null) {
b.setPath(info.getPath().toUri().getPath());
}
if (info.getReplication() != null) {
b.setReplication(info.getReplication());
}
if (info.getPool() != null) {
b.setPool(info.getPool());
}
Expiration expiry = info.getExpiration();
if (expiry != null) {
assert (!expiry.isRelative());
b.setExpiration(PBHelper.convert(expiry));
}
directives.add(b.build());
}
CacheManagerSection s = CacheManagerSection.newBuilder()
.setNextDirectiveId(nextDirectiveId).setNumPools(pools.size())
.setNumDirectives(directives.size()).build();
return new PersistState(s, pools, directives);
}
/**
* Reloads CacheManager state from the passed DataInput. Used during namenode
* startup to restore CacheManager state from an FSImage.
@ -954,6 +1032,56 @@ public void loadStateCompat(DataInput in) throws IOException {
serializerCompat.load(in);
}
public void loadState(PersistState s) throws IOException {
nextDirectiveId = s.section.getNextDirectiveId();
for (CachePoolInfoProto p : s.pools) {
CachePoolInfo info = new CachePoolInfo(p.getPoolName());
if (p.hasOwnerName())
info.setOwnerName(p.getOwnerName());
if (p.hasGroupName())
info.setGroupName(p.getGroupName());
if (p.hasMode())
info.setMode(new FsPermission((short) p.getMode()));
if (p.hasLimit())
info.setLimit(p.getLimit());
addCachePool(info);
}
for (CacheDirectiveInfoProto p : s.directives) {
// Get pool reference by looking it up in the map
final String poolName = p.getPool();
CacheDirective directive = new CacheDirective(p.getId(), new Path(
p.getPath()).toUri().getPath(), (short) p.getReplication(), p
.getExpiration().getMillis());
addCacheDirective(poolName, directive);
}
}
private void addCacheDirective(final String poolName,
final CacheDirective directive) throws IOException {
CachePool pool = cachePools.get(poolName);
if (pool == null) {
throw new IOException("Directive refers to pool " + poolName
+ ", which does not exist.");
}
boolean addedDirective = pool.getDirectiveList().add(directive);
assert addedDirective;
if (directivesById.put(directive.getId(), directive) != null) {
throw new IOException("A directive with ID " + directive.getId()
+ " already exists");
}
List<CacheDirective> directives = directivesByPath.get(directive.getPath());
if (directives == null) {
directives = new LinkedList<CacheDirective>();
directivesByPath.put(directive.getPath(), directives);
}
directives.add(directive);
}
private final class SerializerCompat {
private void save(DataOutputStream out, String sdPath) throws IOException {
out.writeLong(nextDirectiveId);
@ -1036,27 +1164,10 @@ private void loadDirectives(DataInput in) throws IOException {
CacheDirectiveInfo info = FSImageSerialization.readCacheDirectiveInfo(in);
// Get pool reference by looking it up in the map
final String poolName = info.getPool();
CachePool pool = cachePools.get(poolName);
if (pool == null) {
throw new IOException("Directive refers to pool " + poolName +
", which does not exist.");
}
CacheDirective directive =
new CacheDirective(info.getId(), info.getPath().toUri().getPath(),
info.getReplication(), info.getExpiration().getAbsoluteMillis());
boolean addedDirective = pool.getDirectiveList().add(directive);
assert addedDirective;
if (directivesById.put(directive.getId(), directive) != null) {
throw new IOException("A directive with ID " + directive.getId() +
" already exists");
}
List<CacheDirective> directives =
directivesByPath.get(directive.getPath());
if (directives == null) {
directives = new LinkedList<CacheDirective>();
directivesByPath.put(directive.getPath(), directives);
}
directives.add(directive);
addCacheDirective(poolName, directive);
counter.increment();
}
prog.endStep(Phase.LOADING_FSIMAGE, step);

View File

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

View File

@ -42,6 +42,12 @@ public interface FSClusterStats {
* for writing targets, and false otherwise.
*/
public boolean isAvoidingStaleDataNodesForWrite();
/**
* Indicates number of datanodes that are in service.
* @return Number of datanodes that are both alive and not decommissioned.
*/
public int getNumDatanodesInService();
}

View File

@ -813,8 +813,7 @@ private void loadFSImage(File imageFile, FSNamesystem target,
*/
private void loadFSImage(File curFile, MD5Hash expectedMd5,
FSNamesystem target, MetaRecoveryContext recovery) throws IOException {
FSImageFormat.Loader loader = new FSImageFormat.Loader(
conf, target);
FSImageFormat.LoaderDelegator loader = FSImageFormat.newLoader(conf, target);
loader.load(curFile);
target.setBlockPoolId(this.getBlockPoolID());
@ -843,7 +842,7 @@ void saveFSImage(SaveNamespaceContext context, StorageDirectory sd)
File newFile = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE_NEW, txid);
File dstFile = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE, txid);
FSImageFormat.Saver saver = new FSImageFormat.Saver(context);
FSImageFormatProtobuf.Saver saver = new FSImageFormatProtobuf.Saver(context);
FSImageCompression compression = FSImageCompression.createCompression(conf);
saver.save(newFile, compression);

View File

@ -57,6 +57,10 @@ private FSImageCompression(CompressionCodec codec) {
imageCodec = codec;
}
public CompressionCodec getImageCodec() {
return imageCodec;
}
/**
* Create a "noop" compression - i.e. uncompressed
*/
@ -89,7 +93,7 @@ static FSImageCompression createCompression(Configuration conf)
* Create a compression instance using the codec specified by
* <code>codecClassName</code>
*/
private static FSImageCompression createCompression(Configuration conf,
static FSImageCompression createCompression(Configuration conf,
String codecClassName)
throws IOException {

View File

@ -68,12 +68,13 @@
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
import org.apache.hadoop.hdfs.util.ReadOnlyList;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.util.StringUtils;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.annotations.VisibleForTesting;
/**
* Contains inner classes for reading or writing the on-disk format for
@ -180,16 +181,74 @@
@InterfaceStability.Evolving
public class FSImageFormat {
private static final Log LOG = FSImage.LOG;
// Static-only class
private FSImageFormat() {}
interface AbstractLoader {
MD5Hash getLoadedImageMd5();
long getLoadedImageTxId();
}
static class LoaderDelegator implements AbstractLoader {
private AbstractLoader impl;
private final Configuration conf;
private final FSNamesystem fsn;
LoaderDelegator(Configuration conf, FSNamesystem fsn) {
this.conf = conf;
this.fsn = fsn;
}
@Override
public MD5Hash getLoadedImageMd5() {
return impl.getLoadedImageMd5();
}
@Override
public long getLoadedImageTxId() {
return impl.getLoadedImageTxId();
}
public void load(File file) throws IOException {
Preconditions.checkState(impl == null, "Image already loaded!");
FileInputStream is = null;
try {
is = new FileInputStream(file);
byte[] magic = new byte[FSImageUtil.MAGIC_HEADER.length];
IOUtils.readFully(is, magic, 0, magic.length);
if (Arrays.equals(magic, FSImageUtil.MAGIC_HEADER)) {
FSImageFormatProtobuf.Loader loader = new FSImageFormatProtobuf.Loader(
conf, fsn);
impl = loader;
loader.load(file);
} else {
Loader loader = new Loader(conf, fsn);
impl = loader;
loader.load(file);
}
} finally {
IOUtils.cleanup(LOG, is);
}
}
}
/**
* Construct a loader class to load the image. It chooses the loader based on
* the layout version.
*/
public static LoaderDelegator newLoader(Configuration conf, FSNamesystem fsn) {
return new LoaderDelegator(conf, fsn);
}
/**
* A one-shot class responsible for loading an image. The load() function
* should be called once, after which the getter methods may be used to retrieve
* information about the image that was loaded, if loading was successful.
*/
public static class Loader {
public static class Loader implements AbstractLoader {
private final Configuration conf;
/** which namesystem this loader is working for */
private final FSNamesystem namesystem;
@ -214,12 +273,14 @@ public static class Loader {
* Return the MD5 checksum of the image that has been loaded.
* @throws IllegalStateException if load() has not yet been called.
*/
MD5Hash getLoadedImageMd5() {
@Override
public MD5Hash getLoadedImageMd5() {
checkLoaded();
return imgDigest;
}
long getLoadedImageTxId() {
@Override
public long getLoadedImageTxId() {
checkLoaded();
return imgTxId;
}
@ -242,7 +303,7 @@ private void checkNotLoaded() {
}
}
void load(File curFile) throws IOException {
public void load(File curFile) throws IOException {
checkNotLoaded();
assert curFile != null : "curFile is null";

View File

@ -0,0 +1,426 @@
/**
* 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.server.namenode;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FilesUnderConstructionSection.FileUnderConstructionEntry;
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.snapshot.Snapshot;
import org.apache.hadoop.hdfs.util.ReadOnlyList;
import com.google.common.base.Preconditions;
import com.google.protobuf.ByteString;
@InterfaceAudience.Private
public final class FSImageFormatPBINode {
private final static long USER_GROUP_STRID_MASK = (1 << 24) - 1;
private final static int USER_STRID_OFFSET = 40;
private final static int GROUP_STRID_OFFSET = 16;
private static final Log LOG = LogFactory.getLog(FSImageFormatProtobuf.class);
public final static class Loader {
public static PermissionStatus loadPermission(long id,
final String[] stringTable) {
short perm = (short) (id & ((1 << GROUP_STRID_OFFSET) - 1));
int gsid = (int) ((id >> GROUP_STRID_OFFSET) & USER_GROUP_STRID_MASK);
int usid = (int) ((id >> USER_STRID_OFFSET) & USER_GROUP_STRID_MASK);
return new PermissionStatus(stringTable[usid], stringTable[gsid],
new FsPermission(perm));
}
public static INodeDirectory loadINodeDirectory(INodeSection.INode n,
final String[] stringTable) {
assert n.getType() == INodeSection.INode.Type.DIRECTORY;
INodeSection.INodeDirectory d = n.getDirectory();
final PermissionStatus permissions = loadPermission(d.getPermission(),
stringTable);
final INodeDirectory dir = new INodeDirectory(n.getId(), n.getName()
.toByteArray(), permissions, d.getModificationTime());
final long nsQuota = d.getNsQuota(), dsQuota = d.getDsQuota();
if (nsQuota >= 0 || dsQuota >= 0) {
dir.addDirectoryWithQuotaFeature(nsQuota, dsQuota);
}
return dir;
}
public static void updateBlocksMap(INodeFile file, BlockManager bm) {
// Add file->block mapping
final BlockInfo[] blocks = file.getBlocks();
if (blocks != null) {
for (int i = 0; i < blocks.length; i++) {
file.setBlock(i, bm.addBlockCollection(blocks[i], file));
}
}
}
private final FSDirectory dir;
private final FSNamesystem fsn;
private final FSImageFormatProtobuf.Loader parent;
Loader(FSNamesystem fsn, final FSImageFormatProtobuf.Loader parent) {
this.fsn = fsn;
this.dir = fsn.dir;
this.parent = parent;
}
void loadINodeDirectorySection(InputStream in) throws IOException {
final List<INodeReference> refList = parent.getLoaderContext()
.getRefList();
while (true) {
INodeDirectorySection.DirEntry e = INodeDirectorySection.DirEntry
.parseDelimitedFrom(in);
// note that in is a LimitedInputStream
if (e == null) {
break;
}
INodeDirectory p = dir.getInode(e.getParent()).asDirectory();
for (long id : e.getChildrenList()) {
INode child = dir.getInode(id);
addToParent(p, child);
}
for (int refId : e.getRefChildrenList()) {
INodeReference ref = refList.get(refId);
addToParent(p, ref);
}
}
}
void loadINodeSection(InputStream in) throws IOException {
INodeSection s = INodeSection.parseDelimitedFrom(in);
fsn.resetLastInodeId(s.getLastInodeId());
LOG.info("Loading " + s.getNumInodes() + " INodes.");
for (int i = 0; i < s.getNumInodes(); ++i) {
INodeSection.INode p = INodeSection.INode.parseDelimitedFrom(in);
if (p.getId() == INodeId.ROOT_INODE_ID) {
loadRootINode(p);
} else {
INode n = loadINode(p);
dir.addToInodeMap(n);
}
}
}
/**
* Load the under-construction files section, and update the lease map
*/
void loadFilesUnderConstructionSection(InputStream in) throws IOException {
while (true) {
FileUnderConstructionEntry entry = FileUnderConstructionEntry
.parseDelimitedFrom(in);
if (entry == null) {
break;
}
// update the lease manager
INodeFile file = dir.getInode(entry.getInodeId()).asFile();
FileUnderConstructionFeature uc = file.getFileUnderConstructionFeature();
Preconditions.checkState(uc != null); // file must be under-construction
fsn.leaseManager.addLease(uc.getClientName(), entry.getFullPath());
}
}
private void addToParent(INodeDirectory parent, INode child) {
if (parent == dir.rootDir && FSDirectory.isReservedName(child)) {
throw new HadoopIllegalArgumentException("File name \""
+ child.getLocalName() + "\" is reserved. Please "
+ " change the name of the existing file or directory to another "
+ "name before upgrading to this release.");
}
// NOTE: This does not update space counts for parents
if (!parent.addChild(child)) {
return;
}
dir.cacheName(child);
if (child.isFile()) {
updateBlocksMap(child.asFile(), fsn.getBlockManager());
}
}
private INode loadINode(INodeSection.INode n) {
switch (n.getType()) {
case FILE:
return loadINodeFile(n);
case DIRECTORY:
return loadINodeDirectory(n, parent.getLoaderContext().getStringTable());
case SYMLINK:
return loadINodeSymlink(n);
default:
break;
}
return null;
}
private INodeFile loadINodeFile(INodeSection.INode n) {
assert n.getType() == INodeSection.INode.Type.FILE;
INodeSection.INodeFile f = n.getFile();
List<BlockProto> bp = f.getBlocksList();
short replication = (short) f.getReplication();
BlockInfo[] blocks = new BlockInfo[bp.size()];
for (int i = 0, e = bp.size(); i < e; ++i) {
blocks[i] = new BlockInfo(PBHelper.convert(bp.get(i)), replication);
}
final PermissionStatus permissions = loadPermission(f.getPermission(),
parent.getLoaderContext().getStringTable());
final INodeFile file = new INodeFile(n.getId(),
n.getName().toByteArray(), permissions, f.getModificationTime(),
f.getAccessTime(), blocks, replication, f.getPreferredBlockSize());
// under-construction information
if (f.hasFileUC()) {
INodeSection.FileUnderConstructionFeature uc = f.getFileUC();
file.toUnderConstruction(uc.getClientName(), uc.getClientMachine(),
null);
if (blocks.length > 0) {
BlockInfo lastBlk = file.getLastBlock();
// replace the last block of file
file.setBlock(file.numBlocks() - 1, new BlockInfoUnderConstruction(
lastBlk, replication));
}
}
return file;
}
private INodeSymlink loadINodeSymlink(INodeSection.INode n) {
assert n.getType() == INodeSection.INode.Type.SYMLINK;
INodeSection.INodeSymlink s = n.getSymlink();
final PermissionStatus permissions = loadPermission(s.getPermission(),
parent.getLoaderContext().getStringTable());
return new INodeSymlink(n.getId(), n.getName().toByteArray(), permissions,
0, 0, s.getTarget().toStringUtf8());
}
private void loadRootINode(INodeSection.INode p) {
INodeDirectory root = loadINodeDirectory(p, parent.getLoaderContext()
.getStringTable());
final Quota.Counts q = root.getQuotaCounts();
final long nsQuota = q.get(Quota.NAMESPACE);
final long dsQuota = q.get(Quota.DISKSPACE);
if (nsQuota != -1 || dsQuota != -1) {
dir.rootDir.getDirectoryWithQuotaFeature().setQuota(nsQuota, dsQuota);
}
dir.rootDir.cloneModificationTime(root);
dir.rootDir.clonePermissionStatus(root);
}
}
public final static class Saver {
private static long buildPermissionStatus(INodeAttributes n,
final SaverContext.DeduplicationMap<String> stringMap) {
long userId = stringMap.getId(n.getUserName());
long groupId = stringMap.getId(n.getGroupName());
return ((userId & USER_GROUP_STRID_MASK) << USER_STRID_OFFSET)
| ((groupId & USER_GROUP_STRID_MASK) << GROUP_STRID_OFFSET)
| n.getFsPermissionShort();
}
public static INodeSection.INodeFile.Builder buildINodeFile(
INodeFileAttributes file,
final SaverContext.DeduplicationMap<String> stringMap) {
INodeSection.INodeFile.Builder b = INodeSection.INodeFile.newBuilder()
.setAccessTime(file.getAccessTime())
.setModificationTime(file.getModificationTime())
.setPermission(buildPermissionStatus(file, stringMap))
.setPreferredBlockSize(file.getPreferredBlockSize())
.setReplication(file.getFileReplication());
return b;
}
public static INodeSection.INodeDirectory.Builder buildINodeDirectory(
INodeDirectoryAttributes dir,
final SaverContext.DeduplicationMap<String> stringMap) {
Quota.Counts quota = dir.getQuotaCounts();
INodeSection.INodeDirectory.Builder b = INodeSection.INodeDirectory
.newBuilder().setModificationTime(dir.getModificationTime())
.setNsQuota(quota.get(Quota.NAMESPACE))
.setDsQuota(quota.get(Quota.DISKSPACE))
.setPermission(buildPermissionStatus(dir, stringMap));
return b;
}
private final FSNamesystem fsn;
private final FileSummary.Builder summary;
private final SaveNamespaceContext context;
private final FSImageFormatProtobuf.Saver parent;
Saver(FSImageFormatProtobuf.Saver parent, FileSummary.Builder summary) {
this.parent = parent;
this.summary = summary;
this.context = parent.getContext();
this.fsn = context.getSourceNamesystem();
}
void serializeINodeDirectorySection(OutputStream out) throws IOException {
Iterator<INodeWithAdditionalFields> iter = fsn.getFSDirectory()
.getINodeMap().getMapIterator();
final ArrayList<INodeReference> refList = parent.getSaverContext()
.getRefList();
int i = 0;
while (iter.hasNext()) {
INodeWithAdditionalFields n = iter.next();
if (!n.isDirectory()) {
continue;
}
ReadOnlyList<INode> children = n.asDirectory().getChildrenList(
Snapshot.CURRENT_STATE_ID);
if (children.size() > 0) {
INodeDirectorySection.DirEntry.Builder b = INodeDirectorySection.
DirEntry.newBuilder().setParent(n.getId());
for (INode inode : children) {
if (!inode.isReference()) {
b.addChildren(inode.getId());
} else {
refList.add(inode.asReference());
b.addRefChildren(refList.size() - 1);
}
}
INodeDirectorySection.DirEntry e = b.build();
e.writeDelimitedTo(out);
}
++i;
if (i % FSImageFormatProtobuf.Saver.CHECK_CANCEL_INTERVAL == 0) {
context.checkCancelled();
}
}
parent.commitSection(summary,
FSImageFormatProtobuf.SectionName.INODE_DIR);
}
void serializeINodeSection(OutputStream out) throws IOException {
INodeMap inodesMap = fsn.dir.getINodeMap();
INodeSection.Builder b = INodeSection.newBuilder()
.setLastInodeId(fsn.getLastInodeId()).setNumInodes(inodesMap.size());
INodeSection s = b.build();
s.writeDelimitedTo(out);
int i = 0;
Iterator<INodeWithAdditionalFields> iter = inodesMap.getMapIterator();
while (iter.hasNext()) {
INodeWithAdditionalFields n = iter.next();
save(out, n);
++i;
if (i % FSImageFormatProtobuf.Saver.CHECK_CANCEL_INTERVAL == 0) {
context.checkCancelled();
}
}
parent.commitSection(summary, FSImageFormatProtobuf.SectionName.INODE);
}
void serializeFilesUCSection(OutputStream out) throws IOException {
Map<String, INodeFile> ucMap = fsn.getFilesUnderConstruction();
for (Map.Entry<String, INodeFile> entry : ucMap.entrySet()) {
String path = entry.getKey();
INodeFile file = entry.getValue();
FileUnderConstructionEntry.Builder b = FileUnderConstructionEntry
.newBuilder().setInodeId(file.getId()).setFullPath(path);
FileUnderConstructionEntry e = b.build();
e.writeDelimitedTo(out);
}
parent.commitSection(summary,
FSImageFormatProtobuf.SectionName.FILES_UNDERCONSTRUCTION);
}
private void save(OutputStream out, INode n) throws IOException {
if (n.isDirectory()) {
save(out, n.asDirectory());
} else if (n.isFile()) {
save(out, n.asFile());
} else if (n.isSymlink()) {
save(out, n.asSymlink());
}
}
private void save(OutputStream out, INodeDirectory n) throws IOException {
INodeSection.INodeDirectory.Builder b = buildINodeDirectory(n,
parent.getSaverContext().getStringMap());
INodeSection.INode r = buildINodeCommon(n)
.setType(INodeSection.INode.Type.DIRECTORY).setDirectory(b).build();
r.writeDelimitedTo(out);
}
private void save(OutputStream out, INodeFile n) throws IOException {
INodeSection.INodeFile.Builder b = buildINodeFile(n,
parent.getSaverContext().getStringMap());
for (Block block : n.getBlocks()) {
b.addBlocks(PBHelper.convert(block));
}
FileUnderConstructionFeature uc = n.getFileUnderConstructionFeature();
if (uc != null) {
INodeSection.FileUnderConstructionFeature f =
INodeSection.FileUnderConstructionFeature
.newBuilder().setClientName(uc.getClientName())
.setClientMachine(uc.getClientMachine()).build();
b.setFileUC(f);
}
INodeSection.INode r = buildINodeCommon(n)
.setType(INodeSection.INode.Type.FILE).setFile(b).build();
r.writeDelimitedTo(out);
}
private void save(OutputStream out, INodeSymlink n) throws IOException {
INodeSection.INodeSymlink.Builder b = INodeSection.INodeSymlink
.newBuilder()
.setPermission(buildPermissionStatus(n, parent.getSaverContext().getStringMap()))
.setTarget(ByteString.copyFrom(n.getSymlink()));
INodeSection.INode r = buildINodeCommon(n)
.setType(INodeSection.INode.Type.SYMLINK).setSymlink(b).build();
r.writeDelimitedTo(out);
}
private final INodeSection.INode.Builder buildINodeCommon(INode n) {
return INodeSection.INode.newBuilder()
.setId(n.getId())
.setName(ByteString.copyFrom(n.getLocalNameBytes()));
}
}
private FSImageFormatPBINode() {
}
}

View File

@ -0,0 +1,583 @@
/**
* 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.server.namenode;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.security.DigestOutputStream;
import java.security.MessageDigest;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.CacheManagerSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
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.StringTableSection;
import org.apache.hadoop.hdfs.server.namenode.snapshot.FSImageFormatPBSnapshot;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.CompressorStream;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.io.LimitInputStream;
import com.google.protobuf.CodedOutputStream;
/**
* Utility class to read / write fsimage in protobuf format.
*/
@InterfaceAudience.Private
public final class FSImageFormatProtobuf {
private static final Log LOG = LogFactory.getLog(FSImageFormatProtobuf.class);
public static final class LoaderContext {
private String[] stringTable;
private final ArrayList<INodeReference> refList = Lists.newArrayList();
public String[] getStringTable() {
return stringTable;
}
public ArrayList<INodeReference> getRefList() {
return refList;
}
}
public static final class SaverContext {
public static class DeduplicationMap<E> {
private final Map<E, Integer> map = Maps.newHashMap();
private DeduplicationMap() {}
static <T> DeduplicationMap<T> newMap() {
return new DeduplicationMap<T>();
}
int getId(E value) {
if (value == null) {
return 0;
}
Integer v = map.get(value);
if (v == null) {
int nv = map.size() + 1;
map.put(value, nv);
return nv;
}
return v;
}
int size() {
return map.size();
}
Set<Entry<E, Integer>> entrySet() {
return map.entrySet();
}
}
private final DeduplicationMap<String> stringMap = DeduplicationMap.newMap();
private final ArrayList<INodeReference> refList = Lists.newArrayList();
public DeduplicationMap<String> getStringMap() {
return stringMap;
}
public ArrayList<INodeReference> getRefList() {
return refList;
}
}
public static final class Loader implements FSImageFormat.AbstractLoader {
static final int MINIMUM_FILE_LENGTH = 8;
private final Configuration conf;
private final FSNamesystem fsn;
private final LoaderContext ctx;
/** The MD5 sum of the loaded file */
private MD5Hash imgDigest;
/** The transaction ID of the last edit represented by the loaded file */
private long imgTxId;
Loader(Configuration conf, FSNamesystem fsn) {
this.conf = conf;
this.fsn = fsn;
this.ctx = new LoaderContext();
}
@Override
public MD5Hash getLoadedImageMd5() {
return imgDigest;
}
@Override
public long getLoadedImageTxId() {
return imgTxId;
}
public LoaderContext getLoaderContext() {
return ctx;
}
void load(File file) throws IOException {
long start = System.currentTimeMillis();
imgDigest = MD5FileUtils.computeMd5ForFile(file);
RandomAccessFile raFile = new RandomAccessFile(file, "r");
FileInputStream fin = new FileInputStream(file);
try {
loadInternal(raFile, fin);
long end = System.currentTimeMillis();
LOG.info("Loaded FSImage in " + (end - start) / 1000 + " seconds.");
} finally {
fin.close();
raFile.close();
}
}
private void loadInternal(RandomAccessFile raFile, FileInputStream fin)
throws IOException {
if (!FSImageUtil.checkFileFormat(raFile)) {
throw new IOException("Unrecognized file format");
}
FileSummary summary = FSImageUtil.loadSummary(raFile);
FileChannel channel = fin.getChannel();
FSImageFormatPBINode.Loader inodeLoader = new FSImageFormatPBINode.Loader(
fsn, this);
FSImageFormatPBSnapshot.Loader snapshotLoader = new FSImageFormatPBSnapshot.Loader(
fsn, this);
ArrayList<FileSummary.Section> sections = Lists.newArrayList(summary
.getSectionsList());
Collections.sort(sections, new Comparator<FileSummary.Section>() {
@Override
public int compare(FileSummary.Section s1, FileSummary.Section s2) {
SectionName n1 = SectionName.fromString(s1.getName());
SectionName n2 = SectionName.fromString(s2.getName());
if (n1 == null) {
return n2 == null ? 0 : -1;
} else if (n2 == null) {
return -1;
} else {
return n1.ordinal() - n2.ordinal();
}
}
});
StartupProgress prog = NameNode.getStartupProgress();
/**
* beginStep() and the endStep() calls do not match the boundary of the
* sections. This is because that the current implementation only allows
* a particular step to be started for once.
*/
Step currentStep = null;
for (FileSummary.Section s : sections) {
channel.position(s.getOffset());
InputStream in = new BufferedInputStream(new LimitInputStream(fin,
s.getLength()));
in = FSImageUtil.wrapInputStreamForCompression(conf,
summary.getCodec(), in);
String n = s.getName();
switch (SectionName.fromString(n)) {
case NS_INFO:
loadNameSystemSection(in);
break;
case STRING_TABLE:
loadStringTableSection(in);
break;
case INODE: {
currentStep = new Step(StepType.INODES);
prog.beginStep(Phase.LOADING_FSIMAGE, currentStep);
inodeLoader.loadINodeSection(in);
}
break;
case INODE_REFRENCE:
snapshotLoader.loadINodeReferenceSection(in);
break;
case INODE_DIR:
inodeLoader.loadINodeDirectorySection(in);
break;
case FILES_UNDERCONSTRUCTION:
inodeLoader.loadFilesUnderConstructionSection(in);
break;
case SNAPSHOT:
snapshotLoader.loadSnapshotSection(in);
break;
case SNAPSHOT_DIFF:
snapshotLoader.loadSnapshotDiffSection(in);
break;
case SECRET_MANAGER: {
prog.endStep(Phase.LOADING_FSIMAGE, currentStep);
Step step = new Step(StepType.DELEGATION_TOKENS);
prog.beginStep(Phase.LOADING_FSIMAGE, step);
loadSecretManagerSection(in);
prog.endStep(Phase.LOADING_FSIMAGE, step);
}
break;
case CACHE_MANAGER: {
Step step = new Step(StepType.CACHE_POOLS);
prog.beginStep(Phase.LOADING_FSIMAGE, step);
loadCacheManagerSection(in);
prog.endStep(Phase.LOADING_FSIMAGE, step);
}
break;
default:
LOG.warn("Unregconized section " + n);
break;
}
}
}
private void loadNameSystemSection(InputStream in) throws IOException {
NameSystemSection s = NameSystemSection.parseDelimitedFrom(in);
fsn.setGenerationStampV1(s.getGenstampV1());
fsn.setGenerationStampV2(s.getGenstampV2());
fsn.setGenerationStampV1Limit(s.getGenstampV1Limit());
fsn.setLastAllocatedBlockId(s.getLastAllocatedBlockId());
imgTxId = s.getTransactionId();
}
private void loadStringTableSection(InputStream in) throws IOException {
StringTableSection s = StringTableSection.parseDelimitedFrom(in);
ctx.stringTable = new String[s.getNumEntry() + 1];
for (int i = 0; i < s.getNumEntry(); ++i) {
StringTableSection.Entry e = StringTableSection.Entry
.parseDelimitedFrom(in);
ctx.stringTable[e.getId()] = e.getStr();
}
}
private void loadSecretManagerSection(InputStream in) throws IOException {
SecretManagerSection s = SecretManagerSection.parseDelimitedFrom(in);
int numKeys = s.getNumKeys(), numTokens = s.getNumTokens();
ArrayList<SecretManagerSection.DelegationKey> keys = Lists
.newArrayListWithCapacity(numKeys);
ArrayList<SecretManagerSection.PersistToken> tokens = Lists
.newArrayListWithCapacity(numTokens);
for (int i = 0; i < numKeys; ++i)
keys.add(SecretManagerSection.DelegationKey.parseDelimitedFrom(in));
for (int i = 0; i < numTokens; ++i)
tokens.add(SecretManagerSection.PersistToken.parseDelimitedFrom(in));
fsn.loadSecretManagerState(s, keys, tokens);
}
private void loadCacheManagerSection(InputStream in) throws IOException {
CacheManagerSection s = CacheManagerSection.parseDelimitedFrom(in);
ArrayList<CachePoolInfoProto> pools = Lists.newArrayListWithCapacity(s
.getNumPools());
ArrayList<CacheDirectiveInfoProto> directives = Lists
.newArrayListWithCapacity(s.getNumDirectives());
for (int i = 0; i < s.getNumPools(); ++i)
pools.add(CachePoolInfoProto.parseDelimitedFrom(in));
for (int i = 0; i < s.getNumDirectives(); ++i)
directives.add(CacheDirectiveInfoProto.parseDelimitedFrom(in));
fsn.getCacheManager().loadState(
new CacheManager.PersistState(s, pools, directives));
}
}
public static final class Saver {
public static final int CHECK_CANCEL_INTERVAL = 4096;
private final SaveNamespaceContext context;
private final SaverContext saverContext;
private long currentOffset = FSImageUtil.MAGIC_HEADER.length;
private MD5Hash savedDigest;
private FileChannel fileChannel;
// OutputStream for the section data
private OutputStream sectionOutputStream;
private CompressionCodec codec;
private OutputStream underlyingOutputStream;
Saver(SaveNamespaceContext context) {
this.context = context;
this.saverContext = new SaverContext();
}
public MD5Hash getSavedDigest() {
return savedDigest;
}
public SaveNamespaceContext getContext() {
return context;
}
public SaverContext getSaverContext() {
return saverContext;
}
public void commitSection(FileSummary.Builder summary, SectionName name)
throws IOException {
long oldOffset = currentOffset;
flushSectionOutputStream();
if (codec != null) {
sectionOutputStream = codec.createOutputStream(underlyingOutputStream);
} else {
sectionOutputStream = underlyingOutputStream;
}
long length = fileChannel.position() - oldOffset;
summary.addSections(FileSummary.Section.newBuilder().setName(name.name)
.setLength(length).setOffset(currentOffset));
currentOffset += length;
}
private void flushSectionOutputStream() throws IOException {
if (codec != null) {
((CompressorStream) sectionOutputStream).finish();
}
sectionOutputStream.flush();
}
void save(File file, FSImageCompression compression) throws IOException {
FileOutputStream fout = new FileOutputStream(file);
fileChannel = fout.getChannel();
try {
saveInternal(fout, compression, file.getAbsolutePath().toString());
} finally {
fout.close();
}
}
private static void saveFileSummary(OutputStream out, FileSummary summary)
throws IOException {
summary.writeDelimitedTo(out);
int length = getOndiskTrunkSize(summary);
byte[] lengthBytes = new byte[4];
ByteBuffer.wrap(lengthBytes).asIntBuffer().put(length);
out.write(lengthBytes);
}
private void saveInodes(FileSummary.Builder summary) throws IOException {
FSImageFormatPBINode.Saver saver = new FSImageFormatPBINode.Saver(this,
summary);
saver.serializeINodeSection(sectionOutputStream);
saver.serializeINodeDirectorySection(sectionOutputStream);
saver.serializeFilesUCSection(sectionOutputStream);
}
private void saveSnapshots(FileSummary.Builder summary) throws IOException {
FSImageFormatPBSnapshot.Saver snapshotSaver = new FSImageFormatPBSnapshot.Saver(
this, summary, context, context.getSourceNamesystem());
snapshotSaver.serializeSnapshotSection(sectionOutputStream);
snapshotSaver.serializeSnapshotDiffSection(sectionOutputStream);
snapshotSaver.serializeINodeReferenceSection(sectionOutputStream);
}
private void saveInternal(FileOutputStream fout,
FSImageCompression compression, String filePath) throws IOException {
StartupProgress prog = NameNode.getStartupProgress();
MessageDigest digester = MD5Hash.getDigester();
underlyingOutputStream = new DigestOutputStream(new BufferedOutputStream(
fout), digester);
underlyingOutputStream.write(FSImageUtil.MAGIC_HEADER);
fileChannel = fout.getChannel();
FileSummary.Builder b = FileSummary.newBuilder()
.setOndiskVersion(FSImageUtil.FILE_VERSION)
.setLayoutVersion(LayoutVersion.getCurrentLayoutVersion());
codec = compression.getImageCodec();
if (codec != null) {
b.setCodec(codec.getClass().getCanonicalName());
sectionOutputStream = codec.createOutputStream(underlyingOutputStream);
} else {
sectionOutputStream = underlyingOutputStream;
}
saveNameSystemSection(b);
// Check for cancellation right after serializing the name system section.
// Some unit tests, such as TestSaveNamespace#testCancelSaveNameSpace
// depends on this behavior.
context.checkCancelled();
Step step = new Step(StepType.INODES, filePath);
prog.beginStep(Phase.SAVING_CHECKPOINT, step);
saveInodes(b);
saveSnapshots(b);
prog.endStep(Phase.SAVING_CHECKPOINT, step);
step = new Step(StepType.DELEGATION_TOKENS, filePath);
prog.beginStep(Phase.SAVING_CHECKPOINT, step);
saveSecretManagerSection(b);
prog.endStep(Phase.SAVING_CHECKPOINT, step);
step = new Step(StepType.CACHE_POOLS, filePath);
prog.beginStep(Phase.SAVING_CHECKPOINT, step);
saveCacheManagerSection(b);
prog.endStep(Phase.SAVING_CHECKPOINT, step);
saveStringTableSection(b);
// We use the underlyingOutputStream to write the header. Therefore flush
// the buffered stream (which is potentially compressed) first.
flushSectionOutputStream();
FileSummary summary = b.build();
saveFileSummary(underlyingOutputStream, summary);
underlyingOutputStream.close();
savedDigest = new MD5Hash(digester.digest());
}
private void saveSecretManagerSection(FileSummary.Builder summary)
throws IOException {
final FSNamesystem fsn = context.getSourceNamesystem();
DelegationTokenSecretManager.SecretManagerState state = fsn
.saveSecretManagerState();
state.section.writeDelimitedTo(sectionOutputStream);
for (SecretManagerSection.DelegationKey k : state.keys)
k.writeDelimitedTo(sectionOutputStream);
for (SecretManagerSection.PersistToken t : state.tokens)
t.writeDelimitedTo(sectionOutputStream);
commitSection(summary, SectionName.SECRET_MANAGER);
}
private void saveCacheManagerSection(FileSummary.Builder summary)
throws IOException {
final FSNamesystem fsn = context.getSourceNamesystem();
CacheManager.PersistState state = fsn.getCacheManager().saveState();
state.section.writeDelimitedTo(sectionOutputStream);
for (CachePoolInfoProto p : state.pools)
p.writeDelimitedTo(sectionOutputStream);
for (CacheDirectiveInfoProto p : state.directives)
p.writeDelimitedTo(sectionOutputStream);
commitSection(summary, SectionName.CACHE_MANAGER);
}
private void saveNameSystemSection(FileSummary.Builder summary)
throws IOException {
final FSNamesystem fsn = context.getSourceNamesystem();
OutputStream out = sectionOutputStream;
NameSystemSection.Builder b = NameSystemSection.newBuilder()
.setGenstampV1(fsn.getGenerationStampV1())
.setGenstampV1Limit(fsn.getGenerationStampV1Limit())
.setGenstampV2(fsn.getGenerationStampV2())
.setLastAllocatedBlockId(fsn.getLastAllocatedBlockId())
.setTransactionId(context.getTxId());
// We use the non-locked version of getNamespaceInfo here since
// the coordinating thread of saveNamespace already has read-locked
// the namespace for us. If we attempt to take another readlock
// from the actual saver thread, there's a potential of a
// fairness-related deadlock. See the comments on HDFS-2223.
b.setNamespaceId(fsn.unprotectedGetNamespaceInfo().getNamespaceID());
NameSystemSection s = b.build();
s.writeDelimitedTo(out);
commitSection(summary, SectionName.NS_INFO);
}
private void saveStringTableSection(FileSummary.Builder summary)
throws IOException {
OutputStream out = sectionOutputStream;
StringTableSection.Builder b = StringTableSection.newBuilder()
.setNumEntry(saverContext.stringMap.size());
b.build().writeDelimitedTo(out);
for (Entry<String, Integer> e : saverContext.stringMap.entrySet()) {
StringTableSection.Entry.Builder eb = StringTableSection.Entry
.newBuilder().setId(e.getValue()).setStr(e.getKey());
eb.build().writeDelimitedTo(out);
}
commitSection(summary, SectionName.STRING_TABLE);
}
}
/**
* Supported section name. The order of the enum determines the order of
* loading.
*/
public enum SectionName {
NS_INFO("NS_INFO"),
STRING_TABLE("STRING_TABLE"),
INODE("INODE"),
INODE_REFRENCE("INODE_REFRENCE"),
SNAPSHOT("SNAPSHOT"),
INODE_DIR("INODE_DIR"),
FILES_UNDERCONSTRUCTION("FILES_UNDERCONSTRUCTION"),
SNAPSHOT_DIFF("SNAPSHOT_DIFF"),
SECRET_MANAGER("SECRET_MANAGER"),
CACHE_MANAGER("CACHE_MANAGER");
private static final SectionName[] values = SectionName.values();
public static SectionName fromString(String name) {
for (SectionName n : values) {
if (n.name.equals(name))
return n;
}
return null;
}
private final String name;
private SectionName(String name) {
this.name = name;
}
}
private static int getOndiskTrunkSize(com.google.protobuf.GeneratedMessage s) {
return CodedOutputStream.computeRawVarint32Size(s.getSerializedSize())
+ s.getSerializedSize();
}
private FSImageFormatProtobuf() {
}
}

View File

@ -0,0 +1,93 @@
/**
* 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.server.namenode;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.RandomAccessFile;
import java.util.Arrays;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.Loader;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
import org.apache.hadoop.io.compress.CompressionCodec;
@InterfaceAudience.Private
public final class FSImageUtil {
public static final byte[] MAGIC_HEADER = "HDFSIMG1".getBytes();
public static final int FILE_VERSION = 1;
public static boolean checkFileFormat(RandomAccessFile file)
throws IOException {
if (file.length() < Loader.MINIMUM_FILE_LENGTH)
return false;
byte[] magic = new byte[MAGIC_HEADER.length];
file.readFully(magic);
if (!Arrays.equals(MAGIC_HEADER, magic))
return false;
return true;
}
public static FileSummary loadSummary(RandomAccessFile file)
throws IOException {
final int FILE_LENGTH_FIELD_SIZE = 4;
long fileLength = file.length();
file.seek(fileLength - FILE_LENGTH_FIELD_SIZE);
int summaryLength = file.readInt();
if (summaryLength <= 0) {
throw new IOException("Negative length of the file");
}
file.seek(fileLength - FILE_LENGTH_FIELD_SIZE - summaryLength);
byte[] summaryBytes = new byte[summaryLength];
file.readFully(summaryBytes);
FileSummary summary = FileSummary
.parseDelimitedFrom(new ByteArrayInputStream(summaryBytes));
if (summary.getOndiskVersion() != FILE_VERSION) {
throw new IOException("Unsupported file version "
+ summary.getOndiskVersion());
}
if (!LayoutVersion.supports(Feature.PROTOBUF_FORMAT,
summary.getLayoutVersion())) {
throw new IOException("Unsupported layout version "
+ summary.getLayoutVersion());
}
return summary;
}
public static InputStream wrapInputStreamForCompression(
Configuration conf, String codec, InputStream in) throws IOException {
if (codec.isEmpty())
return in;
FSImageCompression compression = FSImageCompression.createCompression(
conf, codec);
CompressionCodec imageCodec = compression.getImageCodec();
return imageCodec.createInputStream(in);
}
}

View File

@ -181,6 +181,7 @@
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager.SecretManagerState;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
@ -198,6 +199,8 @@
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirType;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection.PersistToken;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
@ -6006,6 +6009,15 @@ void saveFilesUnderConstruction(DataOutputStream out,
}
}
/**
* @return all the under-construction files in the lease map
*/
Map<String, INodeFile> getFilesUnderConstruction() {
synchronized (leaseManager) {
return leaseManager.getINodesUnderConstruction();
}
}
/**
* Register a Backup name-node, verifying that it belongs
* to the correct namespace, and adding it to the set of
@ -6282,6 +6294,10 @@ void saveSecretManagerStateCompat(DataOutputStream out, String sdPath)
dtSecretManager.saveSecretManagerStateCompat(out, sdPath);
}
SecretManagerState saveSecretManagerState() {
return dtSecretManager.saveSecretManagerState();
}
/**
* @param in load the state of secret manager from input stream
*/
@ -6289,6 +6305,12 @@ void loadSecretManagerStateCompat(DataInput in) throws IOException {
dtSecretManager.loadSecretManagerStateCompat(in);
}
void loadSecretManagerState(SecretManagerSection s,
List<SecretManagerSection.DelegationKey> keys,
List<SecretManagerSection.PersistToken> tokens) throws IOException {
dtSecretManager.loadSecretManagerState(new SecretManagerState(s, keys, tokens));
}
/**
* Log the updateMasterKey operation to edit logs
*
@ -6814,7 +6836,12 @@ public boolean isAvoidingStaleDataNodesForWrite() {
return this.blockManager.getDatanodeManager()
.shouldAvoidStaleDataNodesForWrite();
}
@Override // FSClusterStats
public int getNumDatanodesInService() {
return getNumLiveDataNodes() - getNumDecomLiveDataNodes();
}
public SnapshotManager getSnapshotManager() {
return snapshotManager;
}

View File

@ -171,7 +171,7 @@ private int searchChildren(byte[] name) {
return children == null? -1: Collections.binarySearch(children, name);
}
protected DirectoryWithSnapshotFeature addSnapshotFeature(
public DirectoryWithSnapshotFeature addSnapshotFeature(
DirectoryDiffList diffs) {
Preconditions.checkState(!isWithSnapshot(),
"Directory is already with snapshot");

View File

@ -252,7 +252,7 @@ boolean removeLastBlock(Block oldblock) {
/* Start of Snapshot Feature */
private FileWithSnapshotFeature addSnapshotFeature(FileDiffList diffs) {
public FileWithSnapshotFeature addSnapshotFeature(FileDiffList diffs) {
Preconditions.checkState(!isWithSnapshot(),
"File is already with snapshot");
FileWithSnapshotFeature sf = new FileWithSnapshotFeature(diffs);

View File

@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import java.util.Iterator;
import java.util.List;
import org.apache.hadoop.fs.permission.FsPermission;
@ -46,6 +47,10 @@ static INodeMap newInstance(INodeDirectory rootDir) {
/** Synchronized by external lock. */
private final GSet<INode, INodeWithAdditionalFields> map;
public Iterator<INodeWithAdditionalFields> getMapIterator() {
return map.iterator();
}
private INodeMap(GSet<INode, INodeWithAdditionalFields> map) {
Preconditions.checkArgument(map != null);
this.map = map;

View File

@ -17,39 +17,22 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
import java.io.IOException;
import java.io.PrintStream;
import java.net.InetSocketAddress;
import java.net.URI;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import javax.management.ObjectName;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Trash;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
import org.apache.hadoop.ha.HAServiceStatus;
import org.apache.hadoop.ha.HealthCheckFailedException;
import org.apache.hadoop.ha.ServiceFailedException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Trash;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.apache.hadoop.util.ExitUtil.terminate;
import static org.apache.hadoop.util.ToolRunner.confirmPrompt;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HAUtil;
@ -58,20 +41,11 @@
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.ha.ActiveState;
import org.apache.hadoop.hdfs.server.namenode.ha.BootstrapStandby;
import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState;
import org.apache.hadoop.hdfs.server.namenode.ha.*;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgressMetrics;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.*;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.ipc.StandbyException;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@ -89,10 +63,23 @@
import org.apache.hadoop.util.ServicePlugin;
import org.apache.hadoop.util.StringUtils;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import javax.management.ObjectName;
import java.io.IOException;
import java.io.PrintStream;
import java.net.InetSocketAddress;
import java.net.URI;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.apache.hadoop.util.ExitUtil.terminate;
import static org.apache.hadoop.util.ToolRunner.confirmPrompt;
/**********************************************************
* NameNode serves as both directory namespace manager and
@ -183,8 +170,10 @@ public static enum OperationCategory {
DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY,
DFS_NAMENODE_HTTP_ADDRESS_KEY,
DFS_NAMENODE_HTTPS_ADDRESS_KEY,
DFS_NAMENODE_KEYTAB_FILE_KEY,
DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
DFS_NAMENODE_SECONDARY_HTTPS_ADDRESS_KEY,
DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY,
DFS_NAMENODE_BACKUP_ADDRESS_KEY,
DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY,

View File

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

View File

@ -22,6 +22,7 @@
import java.util.List;
import java.util.concurrent.CountDownLatch;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.util.Canceler;
@ -32,7 +33,8 @@
* allows cancellation, and also is responsible for accumulating
* failed storage directories.
*/
class SaveNamespaceContext {
@InterfaceAudience.Private
public class SaveNamespaceContext {
private final FSNamesystem sourceNamesystem;
private final long txid;
private final List<StorageDirectory> errorSDs =
@ -72,7 +74,7 @@ void markComplete() {
completionLatch.countDown();
}
void checkCancelled() throws SaveNamespaceCancelledException {
public void checkCancelled() throws SaveNamespaceCancelledException {
if (canceller.isCancelled()) {
throw new SaveNamespaceCancelledException(
canceller.getCancellationReason());

View File

@ -244,7 +244,7 @@ private DirectoryDiff(int snapshotId, INodeDirectory dir) {
this.isSnapshotRoot = isSnapshotRoot;
}
ChildrenDiff getChildrenDiff() {
public ChildrenDiff getChildrenDiff() {
return diff;
}
@ -343,6 +343,10 @@ public String toString() {
return super.toString() + " childrenSize=" + childrenSize + ", " + diff;
}
int getChildrenSize() {
return childrenSize;
}
@Override
void write(DataOutput out, ReferenceMap referenceMap) throws IOException {
writeSnapshot(out);

View File

@ -0,0 +1,506 @@
/**
* 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.server.namenode.snapshot;
import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.Loader.loadINodeDirectory;
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.Saver.buildINodeDirectory;
import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.Saver.buildINodeFile;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SectionName;
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.INodeReferenceSection;
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.CreatedListEntry;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotDiffSection.DiffEntry.Type;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotSection;
import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
import org.apache.hadoop.hdfs.server.namenode.INodeMap;
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.SaveNamespaceContext;
import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.Root;
import org.apache.hadoop.hdfs.util.Diff.ListType;
import com.google.common.base.Preconditions;
import com.google.protobuf.ByteString;
@InterfaceAudience.Private
public class FSImageFormatPBSnapshot {
/**
* Loading snapshot related information from protobuf based FSImage
*/
public final static class Loader {
private final FSNamesystem fsn;
private final FSDirectory fsDir;
private final FSImageFormatProtobuf.Loader parent;
private final Map<Integer, Snapshot> snapshotMap;
public Loader(FSNamesystem fsn, FSImageFormatProtobuf.Loader parent) {
this.fsn = fsn;
this.fsDir = fsn.getFSDirectory();
this.snapshotMap = new HashMap<Integer, Snapshot>();
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
* directories into {@link INodeDirectorySnapshottable}.
*
*/
public void loadSnapshotSection(InputStream in) throws IOException {
SnapshotManager sm = fsn.getSnapshotManager();
SnapshotSection section = SnapshotSection.parseDelimitedFrom(in);
int snum = section.getNumSnapshots();
sm.setNumSnapshots(snum);
sm.setSnapshotCounter(section.getSnapshotCounter());
for (long sdirId : section.getSnapshottableDirList()) {
INodeDirectory dir = fsDir.getInode(sdirId).asDirectory();
final INodeDirectorySnapshottable sdir;
if (!dir.isSnapshottable()) {
sdir = new INodeDirectorySnapshottable(dir);
fsDir.addToInodeMap(sdir);
} else {
// dir is root, and admin set root to snapshottable before
sdir = (INodeDirectorySnapshottable) dir;
sdir.setSnapshotQuota(INodeDirectorySnapshottable.SNAPSHOT_LIMIT);
}
sm.addSnapshottable(sdir);
}
loadSnapshots(in, snum);
}
private void loadSnapshots(InputStream in, int size) throws IOException {
for (int i = 0; i < size; i++) {
SnapshotSection.Snapshot pbs = SnapshotSection.Snapshot
.parseDelimitedFrom(in);
INodeDirectory root = loadINodeDirectory(pbs.getRoot(),
parent.getLoaderContext().getStringTable());
int sid = pbs.getSnapshotId();
INodeDirectorySnapshottable parent = (INodeDirectorySnapshottable) fsDir
.getInode(root.getId()).asDirectory();
Snapshot snapshot = new Snapshot(sid, root, parent);
// add the snapshot to parent, since we follow the sequence of
// snapshotsByNames when saving, we do not need to sort when loading
parent.addSnapshot(snapshot);
snapshotMap.put(sid, snapshot);
}
}
/**
* Load the snapshot diff section from fsimage.
*/
public void loadSnapshotDiffSection(InputStream in) throws IOException {
final List<INodeReference> refList = parent.getLoaderContext()
.getRefList();
while (true) {
SnapshotDiffSection.DiffEntry entry = SnapshotDiffSection.DiffEntry
.parseDelimitedFrom(in);
if (entry == null) {
break;
}
long inodeId = entry.getInodeId();
INode inode = fsDir.getInode(inodeId);
SnapshotDiffSection.DiffEntry.Type type = entry.getType();
switch (type) {
case FILEDIFF:
loadFileDiffList(in, inode.asFile(), entry.getNumOfDiff());
break;
case DIRECTORYDIFF:
loadDirectoryDiffList(in, inode.asDirectory(), entry.getNumOfDiff(),
refList);
break;
}
}
}
/** Load FileDiff list for a file with snapshot feature */
private void loadFileDiffList(InputStream in, INodeFile file, int size)
throws IOException {
final FileDiffList diffs = new FileDiffList();
for (int i = 0; i < size; i++) {
SnapshotDiffSection.FileDiff pbf = SnapshotDiffSection.FileDiff
.parseDelimitedFrom(in);
INodeFileAttributes copy = null;
if (pbf.hasSnapshotCopy()) {
INodeSection.INodeFile fileInPb = pbf.getSnapshotCopy();
PermissionStatus permission = loadPermission(
fileInPb.getPermission(), parent.getLoaderContext()
.getStringTable());
copy = new INodeFileAttributes.SnapshotCopy(pbf.getName()
.toByteArray(), permission, fileInPb.getModificationTime(),
fileInPb.getAccessTime(), (short) fileInPb.getReplication(),
fileInPb.getPreferredBlockSize());
}
FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
pbf.getFileSize());
diffs.addFirst(diff);
}
file.addSnapshotFeature(diffs);
}
/** Load the created list in a DirectoryDiff */
private List<INode> loadCreatedList(InputStream in, INodeDirectory dir,
int size) throws IOException {
List<INode> clist = new ArrayList<INode>(size);
for (long c = 0; c < size; c++) {
CreatedListEntry entry = CreatedListEntry.parseDelimitedFrom(in);
INode created = SnapshotFSImageFormat.loadCreated(entry.getName()
.toByteArray(), dir);
clist.add(created);
}
return clist;
}
private void addToDeletedList(INode dnode, INodeDirectory parent) {
dnode.setParent(parent);
if (dnode.isFile()) {
updateBlocksMap(dnode.asFile(), fsn.getBlockManager());
}
}
/**
* Load the deleted list in a DirectoryDiff
*/
private List<INode> loadDeletedList(final List<INodeReference> refList,
InputStream in, INodeDirectory dir, List<Long> deletedNodes,
List<Integer> deletedRefNodes)
throws IOException {
List<INode> dlist = new ArrayList<INode>(deletedRefNodes.size()
+ deletedNodes.size());
// load non-reference inodes
for (long deletedId : deletedNodes) {
INode deleted = fsDir.getInode(deletedId);
dlist.add(deleted);
addToDeletedList(deleted, dir);
}
// load reference nodes in the deleted list
for (int refId : deletedRefNodes) {
INodeReference deletedRef = refList.get(refId);
dlist.add(deletedRef);
addToDeletedList(deletedRef, dir);
}
Collections.sort(dlist, new Comparator<INode>() {
@Override
public int compare(INode n1, INode n2) {
return n1.compareTo(n2.getLocalNameBytes());
}
});
return dlist;
}
/** Load DirectoryDiff list for a directory with snapshot feature */
private void loadDirectoryDiffList(InputStream in, INodeDirectory dir,
int size, final List<INodeReference> refList) throws IOException {
if (!dir.isWithSnapshot()) {
dir.addSnapshotFeature(null);
}
DirectoryDiffList diffs = dir.getDiffs();
for (int i = 0; i < size; i++) {
// load a directory diff
SnapshotDiffSection.DirectoryDiff diffInPb = SnapshotDiffSection.
DirectoryDiff.parseDelimitedFrom(in);
final int snapshotId = diffInPb.getSnapshotId();
final Snapshot snapshot = snapshotMap.get(snapshotId);
int childrenSize = diffInPb.getChildrenSize();
boolean useRoot = diffInPb.getIsSnapshotRoot();
INodeDirectoryAttributes copy = null;
if (useRoot) {
copy = snapshot.getRoot();
} else if (diffInPb.hasSnapshotCopy()) {
INodeSection.INodeDirectory dirCopyInPb = diffInPb.getSnapshotCopy();
final byte[] name = diffInPb.getName().toByteArray();
PermissionStatus permission = loadPermission(
dirCopyInPb.getPermission(), parent.getLoaderContext()
.getStringTable());
long modTime = dirCopyInPb.getModificationTime();
boolean noQuota = dirCopyInPb.getNsQuota() == -1
&& dirCopyInPb.getDsQuota() == -1;
copy = noQuota ? new INodeDirectoryAttributes.SnapshotCopy(name,
permission, modTime)
: new INodeDirectoryAttributes.CopyWithQuota(name, permission,
modTime, dirCopyInPb.getNsQuota(), dirCopyInPb.getDsQuota());
}
// load created list
List<INode> clist = loadCreatedList(in, dir,
diffInPb.getCreatedListSize());
// load deleted list
List<INode> dlist = loadDeletedList(refList, in, dir,
diffInPb.getDeletedINodeList(), diffInPb.getDeletedINodeRefList());
// create the directory diff
DirectoryDiff diff = new DirectoryDiff(snapshotId, copy, null,
childrenSize, clist, dlist, useRoot);
diffs.addFirst(diff);
}
}
}
/**
* Saving snapshot related information to protobuf based FSImage
*/
public final static class Saver {
private final FSNamesystem fsn;
private final FileSummary.Builder headers;
private final FSImageFormatProtobuf.Saver parent;
private final SaveNamespaceContext context;
public Saver(FSImageFormatProtobuf.Saver parent,
FileSummary.Builder headers, SaveNamespaceContext context,
FSNamesystem fsn) {
this.parent = parent;
this.headers = headers;
this.context = context;
this.fsn = fsn;
}
/**
* save all the snapshottable directories and snapshots to fsimage
*/
public void serializeSnapshotSection(OutputStream out) throws IOException {
SnapshotManager sm = fsn.getSnapshotManager();
SnapshotSection.Builder b = SnapshotSection.newBuilder()
.setSnapshotCounter(sm.getSnapshotCounter())
.setNumSnapshots(sm.getNumSnapshots());
INodeDirectorySnapshottable[] snapshottables = sm.getSnapshottableDirs();
for (INodeDirectorySnapshottable sdir : snapshottables) {
b.addSnapshottableDir(sdir.getId());
}
b.build().writeDelimitedTo(out);
int i = 0;
for(INodeDirectorySnapshottable sdir : snapshottables) {
for(Snapshot s : sdir.getSnapshotsByNames()) {
Root sroot = s.getRoot();
SnapshotSection.Snapshot.Builder sb = SnapshotSection.Snapshot
.newBuilder().setSnapshotId(s.getId());
INodeSection.INodeDirectory.Builder db = buildINodeDirectory(sroot,
parent.getSaverContext().getStringMap());
INodeSection.INode r = INodeSection.INode.newBuilder()
.setId(sroot.getId())
.setType(INodeSection.INode.Type.DIRECTORY)
.setName(ByteString.copyFrom(sroot.getLocalNameBytes()))
.setDirectory(db).build();
sb.setRoot(r).build().writeDelimitedTo(out);
i++;
if (i % FSImageFormatProtobuf.Saver.CHECK_CANCEL_INTERVAL == 0) {
context.checkCancelled();
}
}
}
Preconditions.checkState(i == sm.getNumSnapshots());
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
*/
public void serializeSnapshotDiffSection(OutputStream out)
throws IOException {
INodeMap inodesMap = fsn.getFSDirectory().getINodeMap();
final List<INodeReference> refList = parent.getSaverContext()
.getRefList();
int i = 0;
Iterator<INodeWithAdditionalFields> iter = inodesMap.getMapIterator();
while (iter.hasNext()) {
INodeWithAdditionalFields inode = iter.next();
if (inode.isFile()) {
serializeFileDiffList(inode.asFile(), out);
} else if (inode.isDirectory()) {
serializeDirDiffList(inode.asDirectory(), refList, out);
}
++i;
if (i % FSImageFormatProtobuf.Saver.CHECK_CANCEL_INTERVAL == 0) {
context.checkCancelled();
}
}
parent.commitSection(headers,
FSImageFormatProtobuf.SectionName.SNAPSHOT_DIFF);
}
private void serializeFileDiffList(INodeFile file, OutputStream out)
throws IOException {
FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature();
if (sf != null) {
List<FileDiff> diffList = sf.getDiffs().asList();
SnapshotDiffSection.DiffEntry entry = SnapshotDiffSection.DiffEntry
.newBuilder().setInodeId(file.getId()).setType(Type.FILEDIFF)
.setNumOfDiff(diffList.size()).build();
entry.writeDelimitedTo(out);
for (int i = diffList.size() - 1; i >= 0; i--) {
FileDiff diff = diffList.get(i);
SnapshotDiffSection.FileDiff.Builder fb = SnapshotDiffSection.FileDiff
.newBuilder().setSnapshotId(diff.getSnapshotId())
.setFileSize(diff.getFileSize());
INodeFileAttributes copy = diff.snapshotINode;
if (copy != null) {
fb.setName(ByteString.copyFrom(copy.getLocalNameBytes()))
.setSnapshotCopy(buildINodeFile(copy, parent.getSaverContext().getStringMap()));
}
fb.build().writeDelimitedTo(out);
}
}
}
private void saveCreatedList(List<INode> created, OutputStream out)
throws IOException {
// local names of the created list member
for (INode c : created) {
SnapshotDiffSection.CreatedListEntry.newBuilder()
.setName(ByteString.copyFrom(c.getLocalNameBytes())).build()
.writeDelimitedTo(out);
}
}
private void serializeDirDiffList(INodeDirectory dir,
final List<INodeReference> refList, OutputStream out)
throws IOException {
DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
if (sf != null) {
List<DirectoryDiff> diffList = sf.getDiffs().asList();
SnapshotDiffSection.DiffEntry entry = SnapshotDiffSection.DiffEntry
.newBuilder().setInodeId(dir.getId()).setType(Type.DIRECTORYDIFF)
.setNumOfDiff(diffList.size()).build();
entry.writeDelimitedTo(out);
for (int i = diffList.size() - 1; i >= 0; i--) { // reverse order!
DirectoryDiff diff = diffList.get(i);
SnapshotDiffSection.DirectoryDiff.Builder db = SnapshotDiffSection.
DirectoryDiff.newBuilder().setSnapshotId(diff.getSnapshotId())
.setChildrenSize(diff.getChildrenSize())
.setIsSnapshotRoot(diff.isSnapshotRoot());
INodeDirectoryAttributes copy = diff.snapshotINode;
if (!diff.isSnapshotRoot() && copy != null) {
db.setName(ByteString.copyFrom(copy.getLocalNameBytes()))
.setSnapshotCopy(
buildINodeDirectory(copy, parent.getSaverContext().getStringMap()));
}
// process created list and deleted list
List<INode> created = diff.getChildrenDiff()
.getList(ListType.CREATED);
db.setCreatedListSize(created.size());
List<INode> deleted = diff.getChildrenDiff().getList(ListType.DELETED);
for (INode d : deleted) {
if (d.isReference()) {
refList.add(d.asReference());
db.addDeletedINodeRef(refList.size() - 1);
} else {
db.addDeletedINode(d.getId());
}
}
db.build().writeDelimitedTo(out);
saveCreatedList(created, out);
}
}
}
}
private FSImageFormatPBSnapshot(){}
}

View File

@ -27,7 +27,6 @@
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormat.Loader;
import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
@ -137,7 +136,7 @@ private static FileDiff loadFileDiff(FileDiff posterior, DataInput in,
* @param parent The directory that the created list belongs to.
* @return The created node.
*/
private static INode loadCreated(byte[] createdNodeName,
public static INode loadCreated(byte[] createdNodeName,
INodeDirectory parent) throws IOException {
// the INode in the created list should be a reference to another INode
// in posterior SnapshotDiffs or one of the current children
@ -209,11 +208,13 @@ private static List<INode> loadDeletedList(INodeDirectory parent,
/**
* Load snapshots and snapshotQuota for a Snapshottable directory.
* @param snapshottableParent The snapshottable directory for loading.
* @param numSnapshots The number of snapshots that the directory has.
* @param in The {@link DataInput} instance to read.
* @param loader The {@link Loader} instance that this loading procedure is
* using.
*
* @param snapshottableParent
* The snapshottable directory for loading.
* @param numSnapshots
* The number of snapshots that the directory has.
* @param loader
* The loader
*/
public static void loadSnapshotList(
INodeDirectorySnapshottable snapshottableParent, int numSnapshots,
@ -231,10 +232,13 @@ public static void loadSnapshotList(
/**
* Load the {@link SnapshotDiff} list for the INodeDirectoryWithSnapshot
* directory.
* @param dir The snapshottable directory for loading.
* @param in The {@link DataInput} instance to read.
* @param loader The {@link Loader} instance that this loading procedure is
* using.
*
* @param dir
* The snapshottable directory for loading.
* @param in
* The {@link DataInput} instance to read.
* @param loader
* The loader
*/
public static void loadDirectoryDiffList(INodeDirectory dir,
DataInput in, FSImageFormat.Loader loader) throws IOException {

View File

@ -270,6 +270,23 @@ public int getNumSnapshots() {
return numSnapshots.get();
}
void setNumSnapshots(int num) {
numSnapshots.set(num);
}
int getSnapshotCounter() {
return snapshotCounter;
}
void setSnapshotCounter(int counter) {
snapshotCounter = counter;
}
INodeDirectorySnapshottable[] getSnapshottableDirs() {
return snapshottables.values().toArray(
new INodeDirectorySnapshottable[snapshottables.size()]);
}
/**
* Write {@link #snapshotCounter}, {@link #numSnapshots},
* and all snapshots to the DataOutput.

View File

@ -107,6 +107,7 @@
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Charsets;
import com.sun.jersey.spi.container.ResourceFilters;
@ -160,9 +161,10 @@ private void init(final UserGroupInformation ugi,
response.setContentType(null);
}
@VisibleForTesting
static DatanodeInfo chooseDatanode(final NameNode namenode,
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();
if (op == PutOpParam.Op.CREATE) {
@ -201,7 +203,7 @@ static DatanodeInfo chooseDatanode(final NameNode namenode,
final LocatedBlocks locations = np.getBlockLocations(path, offset, 1);
final int count = locations.locatedBlockCount();
if (count > 0) {
return JspHelper.bestNode(locations.get(0).getLocations(), false, conf);
return bestNode(locations.get(0).getLocations());
}
}
}
@ -210,13 +212,26 @@ static DatanodeInfo chooseDatanode(final NameNode namenode,
).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(
final NameNode namenode, final UserGroupInformation ugi,
final String renewer) throws IOException {
final Credentials c = DelegationTokenSecretManager.createCredentials(
namenode, ugi, renewer != null? renewer: ugi.getShortUserName());
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);
return t;
}
@ -227,9 +242,8 @@ private URI redirectURI(final NameNode namenode,
final String path, final HttpOpParam.Op op, final long openOffset,
final long blocksize,
final Param<?, ?>... parameters) throws URISyntaxException, IOException {
final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
final DatanodeInfo dn = chooseDatanode(namenode, path, op, openOffset,
blocksize, conf);
blocksize);
final String delegationQuery;
if (!UserGroupInformation.isSecurityEnabled()) {

View File

@ -0,0 +1,160 @@
/**
* 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.tools.offlineImageViewer;
import java.io.BufferedInputStream;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.PrintWriter;
import java.io.RandomAccessFile;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SectionName;
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.INodeSection;
import org.apache.hadoop.io.IOUtils;
import com.google.common.base.Preconditions;
import com.google.common.io.LimitInputStream;
/**
* This is the tool for analyzing file sizes in the namespace image. In order to
* run the tool one should define a range of integers <tt>[0, maxSize]</tt> by
* specifying <tt>maxSize</tt> and a <tt>step</tt>. The range of integers is
* divided into segments of size <tt>step</tt>:
* <tt>[0, s<sub>1</sub>, ..., s<sub>n-1</sub>, maxSize]</tt>, and the visitor
* calculates how many files in the system fall into each segment
* <tt>[s<sub>i-1</sub>, s<sub>i</sub>)</tt>. Note that files larger than
* <tt>maxSize</tt> always fall into the very last segment.
*
* <h3>Input.</h3>
* <ul>
* <li><tt>filename</tt> specifies the location of the image file;</li>
* <li><tt>maxSize</tt> determines the range <tt>[0, maxSize]</tt> of files
* sizes considered by the visitor;</li>
* <li><tt>step</tt> the range is divided into segments of size step.</li>
* </ul>
*
* <h3>Output.</h3> The output file is formatted as a tab separated two column
* table: Size and NumFiles. Where Size represents the start of the segment, and
* numFiles is the number of files form the image which size falls in this
* segment.
*
*/
final class FileDistributionCalculator {
private final static long MAX_SIZE_DEFAULT = 0x2000000000L; // 1/8 TB = 2^37
private final static int INTERVAL_DEFAULT = 0x200000; // 2 MB = 2^21
private final Configuration conf;
private final long maxSize;
private final int steps;
private final PrintWriter out;
private int[] distribution;
private int totalFiles;
private int totalDirectories;
private int totalBlocks;
private long totalSpace;
private long maxFileSize;
FileDistributionCalculator(Configuration conf, long maxSize, int steps,
PrintWriter out) {
this.conf = conf;
this.maxSize = maxSize == 0 ? MAX_SIZE_DEFAULT : maxSize;
this.steps = steps == 0 ? INTERVAL_DEFAULT : steps;
this.out = out;
long numIntervals = this.maxSize / this.steps;
this.distribution = new int[1 + (int) (numIntervals)];
Preconditions.checkState(numIntervals < Integer.MAX_VALUE,
"Too many distribution intervals");
}
void visit(RandomAccessFile file) throws IOException {
if (!FSImageUtil.checkFileFormat(file)) {
throw new IOException("Unrecognized FSImage");
}
FileSummary summary = FSImageUtil.loadSummary(file);
FileInputStream in = null;
try {
in = new FileInputStream(file.getFD());
for (FileSummary.Section s : summary.getSectionsList()) {
if (SectionName.fromString(s.getName()) != SectionName.INODE) {
continue;
}
in.getChannel().position(s.getOffset());
InputStream is = FSImageUtil.wrapInputStreamForCompression(conf,
summary.getCodec(), new BufferedInputStream(new LimitInputStream(
in, s.getLength())));
run(is);
output();
}
} finally {
IOUtils.cleanup(null, in);
}
}
private void run(InputStream in) throws IOException {
INodeSection s = INodeSection.parseDelimitedFrom(in);
for (int i = 0; i < s.getNumInodes(); ++i) {
INodeSection.INode p = INodeSection.INode.parseDelimitedFrom(in);
if (p.getType() == INodeSection.INode.Type.FILE) {
++totalFiles;
INodeSection.INodeFile f = p.getFile();
totalBlocks += f.getBlocksCount();
long fileSize = 0;
for (BlockProto b : f.getBlocksList()) {
fileSize += b.getNumBytes() * f.getReplication();
}
maxFileSize = Math.max(fileSize, maxFileSize);
totalSpace += fileSize;
int bucket = fileSize > maxSize ? distribution.length - 1 : (int) Math
.ceil((double)fileSize / steps);
++distribution[bucket];
} else if (p.getType() == INodeSection.INode.Type.DIRECTORY) {
++totalDirectories;
}
if (i % (1 << 20) == 0) {
out.println("Processed " + i + " inodes.");
}
}
}
private void output() {
// write the distribution into the output file
out.print("Size\tNumFiles\n");
for (int i = 0; i < distribution.length; i++) {
if (distribution[i] != 0) {
out.print(((long) i * steps) + "\t" + distribution[i]);
out.print('\n');
}
}
out.print("totalFiles = " + totalFiles + "\n");
out.print("totalDirectories = " + totalDirectories + "\n");
out.print("totalBlocks = " + totalBlocks + "\n");
out.print("totalSpace = " + totalSpace + "\n");
out.print("maxFileSize = " + maxFileSize + "\n");
}
}

View File

@ -0,0 +1,250 @@
/**
* 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.tools.offlineImageViewer;
import java.io.BufferedInputStream;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.PrintWriter;
import java.io.RandomAccessFile;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SectionName;
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.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.INode;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeDirectory;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeFile;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeSymlink;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.StringTableSection;
import org.apache.hadoop.hdfs.server.namenode.INodeId;
import org.apache.hadoop.io.IOUtils;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.io.LimitInputStream;
/**
* This is the tool for analyzing file sizes in the namespace image. In order to
* run the tool one should define a range of integers <tt>[0, maxSize]</tt> by
* specifying <tt>maxSize</tt> and a <tt>step</tt>. The range of integers is
* divided into segments of size <tt>step</tt>:
* <tt>[0, s<sub>1</sub>, ..., s<sub>n-1</sub>, maxSize]</tt>, and the visitor
* calculates how many files in the system fall into each segment
* <tt>[s<sub>i-1</sub>, s<sub>i</sub>)</tt>. Note that files larger than
* <tt>maxSize</tt> always fall into the very last segment.
*
* <h3>Input.</h3>
* <ul>
* <li><tt>filename</tt> specifies the location of the image file;</li>
* <li><tt>maxSize</tt> determines the range <tt>[0, maxSize]</tt> of files
* sizes considered by the visitor;</li>
* <li><tt>step</tt> the range is divided into segments of size step.</li>
* </ul>
*
* <h3>Output.</h3> The output file is formatted as a tab separated two column
* table: Size and NumFiles. Where Size represents the start of the segment, and
* numFiles is the number of files form the image which size falls in this
* segment.
*
*/
final class LsrPBImage {
private final Configuration conf;
private final PrintWriter out;
private String[] stringTable;
private HashMap<Long, INodeSection.INode> inodes = Maps.newHashMap();
private HashMap<Long, long[]> dirmap = Maps.newHashMap();
private ArrayList<INodeReferenceSection.INodeReference> refList = Lists.newArrayList();
public LsrPBImage(Configuration conf, PrintWriter out) {
this.conf = conf;
this.out = out;
}
public void visit(RandomAccessFile file) throws IOException {
if (!FSImageUtil.checkFileFormat(file)) {
throw new IOException("Unrecognized FSImage");
}
FileSummary summary = FSImageUtil.loadSummary(file);
FileInputStream fin = null;
try {
fin = new FileInputStream(file.getFD());
ArrayList<FileSummary.Section> sections = Lists.newArrayList(summary
.getSectionsList());
Collections.sort(sections, new Comparator<FileSummary.Section>() {
@Override
public int compare(FileSummary.Section s1, FileSummary.Section s2) {
SectionName n1 = SectionName.fromString(s1.getName());
SectionName n2 = SectionName.fromString(s2.getName());
if (n1 == null) {
return n2 == null ? 0 : -1;
} else if (n2 == null) {
return -1;
} else {
return n1.ordinal() - n2.ordinal();
}
}
});
for (FileSummary.Section s : sections) {
fin.getChannel().position(s.getOffset());
InputStream is = FSImageUtil.wrapInputStreamForCompression(conf,
summary.getCodec(), new BufferedInputStream(new LimitInputStream(
fin, s.getLength())));
switch (SectionName.fromString(s.getName())) {
case STRING_TABLE:
loadStringTable(is);
break;
case INODE:
loadINodeSection(is);
break;
case INODE_REFRENCE:
loadINodeReferenceSection(is);
break;
case INODE_DIR:
loadINodeDirectorySection(is);
break;
default:
break;
}
}
list("", INodeId.ROOT_INODE_ID);
} finally {
IOUtils.cleanup(null, fin);
}
}
private void list(String parent, long dirId) {
INode inode = inodes.get(dirId);
listINode(parent.isEmpty() ? "/" : parent, inode);
long[] children = dirmap.get(dirId);
if (children == null) {
return;
}
String newParent = parent + inode.getName().toStringUtf8() + "/";
for (long cid : children) {
list(newParent, cid);
}
}
private void listINode(String parent, INode inode) {
switch (inode.getType()) {
case FILE: {
INodeFile f = inode.getFile();
PermissionStatus p = FSImageFormatPBINode.Loader.loadPermission(
f.getPermission(), stringTable);
out.print(String.format("-%s %2s %8s %10s %10s %10d %s%s\n", p
.getPermission().toString(), f.getReplication(), p.getUserName(), p
.getGroupName(), f.getModificationTime(), getFileSize(f), parent,
inode.getName().toStringUtf8()));
}
break;
case DIRECTORY: {
INodeDirectory d = inode.getDirectory();
PermissionStatus p = FSImageFormatPBINode.Loader.loadPermission(
d.getPermission(), stringTable);
out.print(String.format("d%s - %8s %10s %10s %10d %s%s\n", p
.getPermission().toString(), p.getUserName(), p.getGroupName(), d
.getModificationTime(), 0, parent, inode.getName().toStringUtf8()));
}
break;
case SYMLINK: {
INodeSymlink d = inode.getSymlink();
PermissionStatus p = FSImageFormatPBINode.Loader.loadPermission(
d.getPermission(), stringTable);
out.print(String.format("-%s - %8s %10s %10s %10d %s%s -> %s\n", p
.getPermission().toString(), p.getUserName(), p.getGroupName(), 0, 0,
parent, inode.getName().toStringUtf8(), d.getTarget().toStringUtf8()));
}
break;
default:
break;
}
}
private long getFileSize(INodeFile f) {
long size = 0;
for (BlockProto p : f.getBlocksList()) {
size += p.getNumBytes();
}
return size;
}
private void loadINodeDirectorySection(InputStream in) throws IOException {
while (true) {
INodeDirectorySection.DirEntry e = INodeDirectorySection.DirEntry
.parseDelimitedFrom(in);
// note that in is a LimitedInputStream
if (e == null) {
break;
}
long[] l = new long[e.getChildrenCount() + e.getRefChildrenCount()];
for (int i = 0; i < e.getChildrenCount(); ++i) {
l[i] = e.getChildren(i);
}
for (int i = e.getChildrenCount(); i < l.length; i++) {
int refId = e.getRefChildren(i - e.getChildrenCount());
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);
}
}
private void loadINodeSection(InputStream in) throws IOException {
INodeSection s = INodeSection.parseDelimitedFrom(in);
for (int i = 0; i < s.getNumInodes(); ++i) {
INodeSection.INode p = INodeSection.INode.parseDelimitedFrom(in);
inodes.put(p.getId(), p);
}
}
private void loadStringTable(InputStream in) throws IOException {
StringTableSection s = StringTableSection.parseDelimitedFrom(in);
stringTable = new String[s.getNumEntry() + 1];
for (int i = 0; i < s.getNumEntry(); ++i) {
StringTableSection.Entry e = StringTableSection.Entry
.parseDelimitedFrom(in);
stringTable[e.getId()] = e.getStr();
}
}
}

View File

@ -0,0 +1,178 @@
/**
* 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.tools.offlineImageViewer;
import java.io.EOFException;
import java.io.File;
import java.io.IOException;
import java.io.PrintWriter;
import java.io.RandomAccessFile;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.OptionBuilder;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.cli.PosixParser;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
/**
* OfflineImageViewer to dump the contents of an Hadoop image file to XML or the
* console. Main entry point into utility, either via the command line or
* programatically.
*/
@InterfaceAudience.Private
public class OfflineImageViewerPB {
public static final Log LOG = LogFactory.getLog(OfflineImageViewerPB.class);
private final static String usage = "Usage: bin/hdfs oiv [OPTIONS] -i INPUTFILE -o OUTPUTFILE\n"
+ "Offline Image Viewer\n"
+ "View a Hadoop fsimage INPUTFILE using the specified PROCESSOR,\n"
+ "saving the results in OUTPUTFILE.\n"
+ "\n"
+ "The oiv utility will attempt to parse correctly formed image files\n"
+ "and will abort fail with mal-formed image files.\n"
+ "\n"
+ "The tool works offline and does not require a running cluster in\n"
+ "order to process an image file.\n"
+ "\n"
+ "The following image processors are available:\n"
+ " * Ls: The default image processor generates an lsr-style listing\n"
+ " of the files in the namespace, with the same fields in the same\n"
+ " order. Note that in order to correctly determine file sizes,\n"
+ " this formatter cannot skip blocks and will override the\n"
+ " -skipBlocks option.\n"
+ " * XML: This processor creates an XML document with all elements of\n"
+ " the fsimage enumerated, suitable for further analysis by XML\n"
+ " tools.\n"
+ " * FileDistribution: This processor analyzes the file size\n"
+ " distribution in the image.\n"
+ " -maxSize specifies the range [0, maxSize] of file sizes to be\n"
+ " analyzed (128GB by default).\n"
+ " -step defines the granularity of the distribution. (2MB by default)\n"
+ "\n"
+ "Required command line arguments:\n"
+ "-i,--inputFile <arg> FSImage file to process.\n"
+ "-o,--outputFile <arg> Name of output file. If the specified\n"
+ " file exists, it will be overwritten.\n"
+ "\n"
+ "Optional command line arguments:\n"
+ "-p,--processor <arg> Select which type of processor to apply\n"
+ " against image file."
+ " (Ls|XML|FileDistribution).\n"
+ "-h,--help Display usage information and exit\n";
/**
* Build command-line options and descriptions
*/
private static Options buildOptions() {
Options options = new Options();
// Build in/output file arguments, which are required, but there is no
// addOption method that can specify this
OptionBuilder.isRequired();
OptionBuilder.hasArgs();
OptionBuilder.withLongOpt("outputFile");
options.addOption(OptionBuilder.create("o"));
OptionBuilder.isRequired();
OptionBuilder.hasArgs();
OptionBuilder.withLongOpt("inputFile");
options.addOption(OptionBuilder.create("i"));
options.addOption("p", "processor", true, "");
options.addOption("h", "help", false, "");
options.addOption("skipBlocks", false, "");
options.addOption("printToScreen", false, "");
options.addOption("delimiter", true, "");
return options;
}
/**
* Entry point to command-line-driven operation. User may specify options and
* start fsimage viewer from the command line. Program will process image file
* and exit cleanly or, if an error is encountered, inform user and exit.
*
* @param args
* Command line options
* @throws IOException
*/
public static void main(String[] args) throws IOException {
Options options = buildOptions();
if (args.length == 0) {
printUsage();
return;
}
CommandLineParser parser = new PosixParser();
CommandLine cmd;
try {
cmd = parser.parse(options, args);
} catch (ParseException e) {
System.out.println("Error parsing command-line options: ");
printUsage();
return;
}
if (cmd.hasOption("h")) { // print help and exit
printUsage();
return;
}
String inputFile = cmd.getOptionValue("i");
String processor = cmd.getOptionValue("p", "Ls");
String outputFile = cmd.getOptionValue("o");
PrintWriter out = (outputFile == null || outputFile.equals("-")) ? new PrintWriter(
System.out) : new PrintWriter(new File(outputFile));
Configuration conf = new Configuration();
try {
if (processor.equals("FileDistribution")) {
long maxSize = Long.parseLong(cmd.getOptionValue("maxSize", "0"));
int step = Integer.parseInt(cmd.getOptionValue("step", "0"));
new FileDistributionCalculator(conf, maxSize, step, out)
.visit(new RandomAccessFile(inputFile, "r"));
} else if (processor.equals("XML")) {
new PBImageXmlWriter(conf, out).visit(new RandomAccessFile(inputFile,
"r"));
} else {
new LsrPBImage(conf, out).visit(new RandomAccessFile(inputFile, "r"));
}
} catch (EOFException e) {
System.err.println("Input file ended unexpectedly. Exiting");
} catch (IOException e) {
System.err.println("Encountered exception. Exiting: " + e.getMessage());
} finally {
out.close();
}
}
/**
* Print application usage instructions.
*/
private static void printUsage() {
System.out.println(usage);
}
}

View File

@ -0,0 +1,433 @@
/**
* 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.tools.offlineImageViewer;
import java.io.BufferedInputStream;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.PrintWriter;
import java.io.RandomAccessFile;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SectionName;
import org.apache.hadoop.hdfs.server.namenode.FSImageUtil;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.CacheManagerSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FilesUnderConstructionSection.FileUnderConstructionEntry;
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.INodeDirectory;
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.SecretManagerSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotDiffSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.StringTableSection;
import org.apache.hadoop.io.IOUtils;
import com.google.common.collect.Lists;
import com.google.common.io.LimitInputStream;
/**
* This is the tool for analyzing file sizes in the namespace image. In order to
* run the tool one should define a range of integers <tt>[0, maxSize]</tt> by
* specifying <tt>maxSize</tt> and a <tt>step</tt>. The range of integers is
* divided into segments of size <tt>step</tt>:
* <tt>[0, s<sub>1</sub>, ..., s<sub>n-1</sub>, maxSize]</tt>, and the visitor
* calculates how many files in the system fall into each segment
* <tt>[s<sub>i-1</sub>, s<sub>i</sub>)</tt>. Note that files larger than
* <tt>maxSize</tt> always fall into the very last segment.
*
* <h3>Input.</h3>
* <ul>
* <li><tt>filename</tt> specifies the location of the image file;</li>
* <li><tt>maxSize</tt> determines the range <tt>[0, maxSize]</tt> of files
* sizes considered by the visitor;</li>
* <li><tt>step</tt> the range is divided into segments of size step.</li>
* </ul>
*
* <h3>Output.</h3> The output file is formatted as a tab separated two column
* table: Size and NumFiles. Where Size represents the start of the segment, and
* numFiles is the number of files form the image which size falls in this
* segment.
*
*/
@InterfaceAudience.Private
public final class PBImageXmlWriter {
private final Configuration conf;
private final PrintWriter out;
private String[] stringTable;
public PBImageXmlWriter(Configuration conf, PrintWriter out) {
this.conf = conf;
this.out = out;
}
public void visit(RandomAccessFile file) throws IOException {
if (!FSImageUtil.checkFileFormat(file)) {
throw new IOException("Unrecognized FSImage");
}
FileSummary summary = FSImageUtil.loadSummary(file);
FileInputStream fin = null;
try {
fin = new FileInputStream(file.getFD());
out.print("<?xml version=\"1.0\"?>\n");
ArrayList<FileSummary.Section> sections = Lists.newArrayList(summary
.getSectionsList());
Collections.sort(sections, new Comparator<FileSummary.Section>() {
@Override
public int compare(FileSummary.Section s1, FileSummary.Section s2) {
SectionName n1 = SectionName.fromString(s1.getName());
SectionName n2 = SectionName.fromString(s2.getName());
if (n1 == null) {
return n2 == null ? 0 : -1;
} else if (n2 == null) {
return -1;
} else {
return n1.ordinal() - n2.ordinal();
}
}
});
for (FileSummary.Section s : sections) {
fin.getChannel().position(s.getOffset());
InputStream is = FSImageUtil.wrapInputStreamForCompression(conf,
summary.getCodec(), new BufferedInputStream(new LimitInputStream(
fin, s.getLength())));
switch (SectionName.fromString(s.getName())) {
case NS_INFO:
dumpNameSection(is);
break;
case STRING_TABLE:
loadStringTable(is);
break;
case INODE:
dumpINodeSection(is);
break;
case INODE_REFRENCE:
dumpINodeReferenceSection(is);
break;
case INODE_DIR:
dumpINodeDirectorySection(is);
break;
case FILES_UNDERCONSTRUCTION:
dumpFileUnderConstructionSection(is);
break;
case SNAPSHOT:
dumpSnapshotSection(is);
break;
case SNAPSHOT_DIFF:
dumpSnapshotDiffSection(is);
break;
case SECRET_MANAGER:
dumpSecretManagerSection(is);
break;
case CACHE_MANAGER:
dumpCacheManagerSection(is);
break;
default:
break;
}
}
} finally {
IOUtils.cleanup(null, fin);
}
}
private void dumpCacheManagerSection(InputStream is) throws IOException {
out.print("<CacheManagerSection>");
CacheManagerSection s = CacheManagerSection.parseDelimitedFrom(is);
o("nextDirectiveId", s.getNextDirectiveId());
for (int i = 0; i < s.getNumPools(); ++i) {
CachePoolInfoProto p = CachePoolInfoProto.parseDelimitedFrom(is);
out.print("<pool>");
o("poolName", p.getPoolName()).o("ownerName", p.getOwnerName())
.o("groupName", p.getGroupName()).o("mode", p.getMode())
.o("limit", p.getLimit())
.o("maxRelativeExpiry", p.getMaxRelativeExpiry());
out.print("</pool>\n");
}
for (int i = 0; i < s.getNumPools(); ++i) {
CacheDirectiveInfoProto p = CacheDirectiveInfoProto
.parseDelimitedFrom(is);
out.print("<directive>");
o("id", p.getId()).o("path", p.getPath())
.o("replication", p.getReplication()).o("pool", p.getPool());
out.print("<expiration>");
CacheDirectiveInfoExpirationProto e = p.getExpiration();
o("millis", e.getMillis()).o("relatilve", e.getIsRelative());
out.print("</expiration>\n");
out.print("</directive>\n");
}
out.print("</CacheManagerSection>\n");
}
private void dumpFileUnderConstructionSection(InputStream in)
throws IOException {
out.print("<FileUnderConstructionSection>");
while (true) {
FileUnderConstructionEntry e = FileUnderConstructionEntry
.parseDelimitedFrom(in);
if (e == null) {
break;
}
out.print("<inode>");
o("id", e.getInodeId()).o("path", e.getFullPath());
out.print("</inode>\n");
}
out.print("</FileUnderConstructionSection>\n");
}
private void dumpINodeDirectory(INodeDirectory d) {
o("mtime", d.getModificationTime()).o("permission",
dumpPermission(d.getPermission()));
if (d.hasDsQuota() && d.hasNsQuota()) {
o("nsquota", d.getNsQuota()).o("dsquota", d.getDsQuota());
}
}
private void dumpINodeDirectorySection(InputStream in) throws IOException {
out.print("<INodeDirectorySection>");
while (true) {
INodeDirectorySection.DirEntry e = INodeDirectorySection.DirEntry
.parseDelimitedFrom(in);
// note that in is a LimitedInputStream
if (e == null) {
break;
}
out.print("<directory>");
o("parent", e.getParent());
for (long id : e.getChildrenList()) {
o("inode", id);
}
for (int refId : e.getRefChildrenList()) {
o("inodereference-index", refId);
}
out.print("</directory>\n");
}
out.print("</INodeDirectorySection>\n");
}
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>");
o("referredId", r.getReferredId()).o("name", r.getName().toStringUtf8())
.o("dstSnapshotId", r.getDstSnapshotId())
.o("lastSnapshotId", r.getLastSnapshotId());
out.print("</ref>\n");
}
private void dumpINodeFile(INodeSection.INodeFile f) {
o("replication", f.getReplication()).o("mtime", f.getModificationTime())
.o("atime", f.getAccessTime())
.o("perferredBlockSize", f.getPreferredBlockSize())
.o("permission", dumpPermission(f.getPermission()));
if (f.getBlocksCount() > 0) {
out.print("<blocks>");
for (BlockProto b : f.getBlocksList()) {
out.print("<block>");
o("id", b.getBlockId()).o("genstamp", b.getGenStamp()).o("numBytes",
b.getNumBytes());
out.print("</block>\n");
}
out.print("</blocks>\n");
}
if (f.hasFileUC()) {
INodeSection.FileUnderConstructionFeature u = f.getFileUC();
out.print("<file-under-construction>");
o("clientName", u.getClientName()).o("clientMachine",
u.getClientMachine());
out.print("</file-under-construction>\n");
}
}
private void dumpINodeSection(InputStream in) throws IOException {
INodeSection s = INodeSection.parseDelimitedFrom(in);
out.print("<INodeSection>");
o("lastInodeId", s.getLastInodeId());
for (int i = 0; i < s.getNumInodes(); ++i) {
INodeSection.INode p = INodeSection.INode.parseDelimitedFrom(in);
out.print("<inode>");
o("id", p.getId()).o("type", p.getType()).o("name",
p.getName().toStringUtf8());
if (p.hasFile()) {
dumpINodeFile(p.getFile());
} else if (p.hasDirectory()) {
dumpINodeDirectory(p.getDirectory());
} else if (p.hasSymlink()) {
dumpINodeSymlink(p.getSymlink());
}
out.print("</inode>\n");
}
out.print("</INodeSection>\n");
}
private void dumpINodeSymlink(INodeSymlink s) {
o("permission", dumpPermission(s.getPermission())).o("target",
s.getTarget().toStringUtf8());
}
private void dumpNameSection(InputStream in) throws IOException {
NameSystemSection s = NameSystemSection.parseDelimitedFrom(in);
out.print("<NameSection>\n");
o("genstampV1", s.getGenstampV1()).o("genstampV2", s.getGenstampV2())
.o("genstampV1Limit", s.getGenstampV1Limit())
.o("lastAllocatedBlockId", s.getLastAllocatedBlockId())
.o("txid", s.getTransactionId());
out.print("<NameSection>\n");
}
private String dumpPermission(long permission) {
return FSImageFormatPBINode.Loader.loadPermission(permission, stringTable)
.toString();
}
private void dumpSecretManagerSection(InputStream is) throws IOException {
out.print("<SecretManagerSection>");
SecretManagerSection s = SecretManagerSection.parseDelimitedFrom(is);
o("currentId", s.getCurrentId()).o("tokenSequenceNumber",
s.getTokenSequenceNumber());
out.print("</SecretManagerSection>");
}
private void dumpSnapshotDiffSection(InputStream in) throws IOException {
out.print("<SnapshotDiffSection>");
while (true) {
SnapshotDiffSection.DiffEntry e = SnapshotDiffSection.DiffEntry
.parseDelimitedFrom(in);
if (e == null) {
break;
}
out.print("<diff>");
o("inodeid", e.getInodeId());
switch (e.getType()) {
case FILEDIFF: {
for (int i = 0; i < e.getNumOfDiff(); ++i) {
out.print("<filediff>");
SnapshotDiffSection.FileDiff f = SnapshotDiffSection.FileDiff
.parseDelimitedFrom(in);
o("snapshotId", f.getSnapshotId()).o("size", f.getFileSize()).o(
"name", f.getName().toStringUtf8());
out.print("</filediff>\n");
}
}
break;
case DIRECTORYDIFF: {
for (int i = 0; i < e.getNumOfDiff(); ++i) {
out.print("<dirdiff>");
SnapshotDiffSection.DirectoryDiff d = SnapshotDiffSection.DirectoryDiff
.parseDelimitedFrom(in);
o("snapshotId", d.getSnapshotId())
.o("isSnapshotroot", d.getIsSnapshotRoot())
.o("childrenSize", d.getChildrenSize())
.o("name", d.getName().toStringUtf8());
for (int j = 0; j < d.getCreatedListSize(); ++j) {
SnapshotDiffSection.CreatedListEntry ce = SnapshotDiffSection.CreatedListEntry
.parseDelimitedFrom(in);
out.print("<created>");
o("name", ce.getName().toStringUtf8());
out.print("</created>\n");
}
for (long did : d.getDeletedINodeList()) {
out.print("<deleted>");
o("inode", did);
out.print("</deleted>\n");
}
for (int dRefid : d.getDeletedINodeRefList()) {
out.print("<deleted>");
o("inodereference-index", dRefid);
out.print("</deleted>\n");
}
out.print("</dirdiff>\n");
}
}
break;
default:
break;
}
out.print("</diff>");
}
out.print("<SnapshotDiffSection>\n");
}
private void dumpSnapshotSection(InputStream in) throws IOException {
out.print("<SnapshotSection>");
SnapshotSection s = SnapshotSection.parseDelimitedFrom(in);
o("snapshotCounter", s.getSnapshotCounter());
if (s.getSnapshottableDirCount() > 0) {
out.print("<snapshottableDir>");
for (long id : s.getSnapshottableDirList()) {
o("dir", id);
}
out.print("</snapshottableDir>\n");
}
for (int i = 0; i < s.getNumSnapshots(); ++i) {
SnapshotSection.Snapshot pbs = SnapshotSection.Snapshot
.parseDelimitedFrom(in);
o("snapshot", pbs.getSnapshotId());
}
out.print("</SnapshotSection>\n");
}
private void loadStringTable(InputStream in) throws IOException {
StringTableSection s = StringTableSection.parseDelimitedFrom(in);
stringTable = new String[s.getNumEntry() + 1];
for (int i = 0; i < s.getNumEntry(); ++i) {
StringTableSection.Entry e = StringTableSection.Entry
.parseDelimitedFrom(in);
stringTable[e.getId()] = e.getStr();
}
}
private PBImageXmlWriter o(final String e, final Object v) {
out.print("<" + e + ">" + v + "</" + e + ">");
return this;
}
}

View File

@ -0,0 +1,284 @@
/**
* 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.
*/
option java_package = "org.apache.hadoop.hdfs.server.namenode";
option java_outer_classname = "FsImageProto";
package hadoop.hdfs.fsimage;
import "hdfs.proto";
/**
* This file defines the on-disk layout of the file system image. The
* layout is defined by the following EBNF grammar, in which angle
* brackets mark protobuf definitions. (e.g., <FileSummary>)
*
* FILE := MAGIC SECTION* <FileSummary> FileSummaryLength
* MAGIC := 'HDFSIMG1'
* SECTION := <NameSystemSection> | ...
* FileSummaryLength := 4 byte int
*
* Some notes:
*
* The codec field in FileSummary describes the compression codec used
* for all sections. The fileheader is always uncompressed.
*
* All protobuf messages are serialized in delimited form, which means
* that there always will be an integer indicates the size of the
* protobuf message.
*
*/
message FileSummary {
// The version of the above EBNF grammars.
required uint32 ondiskVersion = 1;
// layoutVersion describes which features are available in the
// FSImage.
required uint32 layoutVersion = 2;
optional string codec = 3;
// index for each section
message Section {
optional string name = 1;
optional uint64 length = 2;
optional uint64 offset = 3;
}
repeated Section sections = 4;
}
/**
* Name: NS_INFO
*/
message NameSystemSection {
optional uint32 namespaceId = 1;
optional uint64 genstampV1 = 2;
optional uint64 genstampV2 = 3;
optional uint64 genstampV1Limit = 4;
optional uint64 lastAllocatedBlockId = 5;
optional uint64 transactionId = 6;
}
/**
* Permission is serialized as a 64-bit long. [0:24):[25:48):[48:64) (in Big Endian).
* The first and the second parts are the string ids of the user and
* group name, and the last 16 bits are the permission bits.
*
* Name: INODE
*/
message INodeSection {
/**
* under-construction feature for INodeFile
*/
message FileUnderConstructionFeature {
optional string clientName = 1;
optional string clientMachine = 2;
}
message INodeFile {
optional uint32 replication = 1;
optional uint64 modificationTime = 2;
optional uint64 accessTime = 3;
optional uint64 preferredBlockSize = 4;
optional fixed64 permission = 5;
repeated BlockProto blocks = 6;
optional FileUnderConstructionFeature fileUC = 7;
}
message INodeDirectory {
optional uint64 modificationTime = 1;
// namespace quota
optional uint64 nsQuota = 2;
// diskspace quota
optional uint64 dsQuota = 3;
optional fixed64 permission = 4;
}
message INodeSymlink {
optional fixed64 permission = 1;
optional bytes target = 2;
}
message INode {
enum Type {
FILE = 1;
DIRECTORY = 2;
SYMLINK = 3;
};
required Type type = 1;
required uint64 id = 2;
optional bytes name = 3;
optional INodeFile file = 4;
optional INodeDirectory directory = 5;
optional INodeSymlink symlink = 6;
}
optional uint64 lastInodeId = 1;
optional uint64 numInodes = 2;
// repeated INodes..
}
/**
* This section records information about under-construction files for
* reconstructing the lease map.
* NAME: FILES_UNDERCONSTRUCTION
*/
message FilesUnderConstructionSection {
message FileUnderConstructionEntry {
optional uint64 inodeId = 1;
optional string fullPath = 2;
}
// repeated FileUnderConstructionEntry...
}
/**
* This section records the children of each directories
* NAME: INODE_DIR
*/
message INodeDirectorySection {
message DirEntry {
optional uint64 parent = 1;
// children that are not reference nodes
repeated uint64 children = 2 [packed = true];
// children that are reference nodes, each element is a reference node id
repeated uint32 refChildren = 3 [packed = true];
}
// 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
* NAME: SNAPSHOT
*/
message SnapshotSection {
message Snapshot {
optional uint32 snapshotId = 1;
// Snapshot root
optional INodeSection.INode root = 2;
}
optional uint32 snapshotCounter = 1;
repeated uint64 snapshottableDir = 2 [packed = true];
// total number of snapshots
optional uint32 numSnapshots = 3;
// repeated Snapshot...
}
/**
* This section records information about snapshot diffs
* NAME: SNAPSHOT_DIFF
*/
message SnapshotDiffSection {
message CreatedListEntry {
optional bytes name = 1;
}
message DirectoryDiff {
optional uint32 snapshotId = 1;
optional uint32 childrenSize = 2;
optional bool isSnapshotRoot = 3;
optional bytes name = 4;
optional INodeSection.INodeDirectory snapshotCopy = 5;
optional uint32 createdListSize = 6;
repeated uint64 deletedINode = 7 [packed = true]; // id of deleted inodes
// id of reference nodes in the deleted list
repeated uint32 deletedINodeRef = 8 [packed = true];
// repeated CreatedListEntry (size is specified by createdListSize)
}
message FileDiff {
optional uint32 snapshotId = 1;
optional uint64 fileSize = 2;
optional bytes name = 3;
optional INodeSection.INodeFile snapshotCopy = 4;
}
message DiffEntry {
enum Type {
FILEDIFF = 1;
DIRECTORYDIFF = 2;
}
required Type type = 1;
optional uint64 inodeId = 2;
optional uint32 numOfDiff = 3;
// repeated DirectoryDiff or FileDiff
}
// repeated DiffEntry
}
/**
* This section maps string to id
* NAME: STRING_TABLE
*/
message StringTableSection {
message Entry {
optional uint32 id = 1;
optional string str = 2;
}
optional uint32 numEntry = 1;
// repeated Entry
}
message SecretManagerSection {
message DelegationKey {
optional uint32 id = 1;
optional uint64 expiryDate = 2;
optional bytes key = 3;
}
message PersistToken {
optional uint32 version = 1;
optional string owner = 2;
optional string renewer = 3;
optional string realUser = 4;
optional uint64 issueDate = 5;
optional uint64 maxDate = 6;
optional uint32 sequenceNumber = 7;
optional uint32 masterKeyId = 8;
optional uint64 expiryDate = 9;
}
optional uint32 currentId = 1;
optional uint32 tokenSequenceNumber = 2;
optional uint32 numKeys = 3;
optional uint32 numTokens = 4;
// repeated DelegationKey keys
// repeated PersistToken tokens
}
message CacheManagerSection {
required uint64 nextDirectiveId = 1;
required uint32 numPools = 2;
required uint32 numDirectives = 3;
// repeated CachePoolInfoProto pools
// repeated CacheDirectiveInfoProto directives
}

View File

@ -96,6 +96,14 @@
</description>
</property>
<property>
<name>dfs.namenode.secondary.https-address</name>
<value>0.0.0.0:50091</value>
<description>
The secondary namenode HTTPS server address and port.
</description>
</property>
<property>
<name>dfs.datanode.address</name>
<value>0.0.0.0:50010</value>
@ -161,6 +169,16 @@
</description>
</property>
<property>
<name>dfs.client.cached.conn.retry</name>
<value>3</value>
<description>The number of times the HDFS client will pull a socket from the
cache. Once this number is exceeded, the client will try to create a new
socket.
</description>
</property>
<property>
<name>dfs.https.server.keystore.resource</name>
<value>ssl-server.xml</value>
@ -1300,7 +1318,16 @@
<name>dfs.journalnode.http-address</name>
<value>0.0.0.0:8480</value>
<description>
The address and port the JournalNode web UI listens on.
The address and port the JournalNode HTTP server listens on.
If the port is 0 then the server will start on a free port.
</description>
</property>
<property>
<name>dfs.journalnode.https-address</name>
<value>0.0.0.0:8481</value>
<description>
The address and port the JournalNode HTTPS server listens on.
If the port is 0 then the server will start on a free port.
</description>
</property>
@ -1489,6 +1516,26 @@
</description>
</property>
<property>
<name>dfs.client.mmap.retry.timeout.ms</name>
<value>300000</value>
<description>
The minimum amount of time that we will wait before retrying a failed mmap
operation.
</description>
</property>
<property>
<name>dfs.client.short.circuit.replica.stale.threshold.ms</name>
<value>3000000</value>
<description>
The maximum amount of time that we will consider a short-circuit replica to
be valid, if there is no communication from the DataNode. After this time
has elapsed, we will re-fetch the short-circuit replica even if it is in
the cache.
</description>
</property>
<property>
<name>dfs.namenode.path.based.cache.block.map.allocation.percent</name>
<value>0.25</value>
@ -1618,4 +1665,15 @@
</description>
</property>
<property>
<name>dfs.client.context</name>
<value>default</value>
<description>
The name of the DFSClient context that we should use. Clients that share
a context share a socket cache and short-circuit cache, among other things.
You should only change this if you don't want to share with another set of
threads.
</description>
</property>
</configuration>

View File

@ -1,3 +1,5 @@
<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
"http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
@ -14,8 +16,6 @@
See the License for the specific language governing permissions and
limitations under the License.
-->
<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
"http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
<html xmlns="http://www.w3.org/1999/xhtml">
<head>
<link rel="stylesheet" type="text/css" href="/static/bootstrap-3.0.2/css/bootstrap.min.css" />

View File

@ -28,7 +28,7 @@
{"name": "nn", "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeInfo"},
{"name": "nnstat", "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus"},
{"name": "fs", "url": "/jmx?qry=Hadoop:service=NameNode,name=FSNamesystemState"},
{"name": "mem", "url": "/jmx?qry=java.lang:type=Memory"},
{"name": "mem", "url": "/jmx?qry=java.lang:type=Memory"}
];
var HELPERS = {
@ -166,14 +166,29 @@
$('#ui-tabs a[href="#tab-snapshot"]').click(load_snapshot_info);
var hash = window.location.hash;
if (hash === "#tab-datanode") {
load_datanode_info();
} else if (hash === "#tab-snapshot") {
load_snapshot_info();
} else if (hash === "#tab-startup-progress") {
load_startup_progress();
} else {
load_overview();
function load_page() {
var hash = window.location.hash;
switch(hash) {
case "#tab-datanode":
load_datanode_info();
break;
case "#tab-snapshot":
load_snapshot_info();
break;
case "#tab-startup-progress":
load_startup_progress();
break;
case "#tab-overview":
load_overview();
break;
default:
window.location.hash = "tab-overview";
break;
}
}
load_page();
$(window).bind('hashchange', function () {
load_page();
});
})();

View File

@ -1,3 +1,5 @@
<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
"http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
@ -14,8 +16,6 @@
See the License for the specific language governing permissions and
limitations under the License.
-->
<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
"http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
<html xmlns="http://www.w3.org/1999/xhtml">
<head>
<link rel="stylesheet" type="text/css" href="/static/bootstrap-3.0.2/css/bootstrap.min.css" />

View File

@ -35,8 +35,8 @@
}
if (sticky) {
var exec = ((parms.perm % 10) & 1) == 1;
res[res.length - 1] = exec ? 't' : 'T';
var otherExec = ((ctx.current().permission % 10) & 1) == 1;
res = res.substr(0, res.length - 1) + (otherExec ? 't' : 'T');
}
chunk.write(dir + res);
@ -52,6 +52,18 @@
$('#alert-panel').show();
}
$(window).bind('hashchange', function () {
$('#alert-panel').hide();
var dir = window.location.hash.slice(1);
if(dir == "") {
dir = "/";
}
if(current_directory != dir) {
browse_directory(dir);
}
});
function network_error_handler(url) {
return function (jqxhr, text, err) {
var msg = '<p>Failed to retreive data from ' + url + ', cause: ' + err + '</p>';
@ -145,6 +157,7 @@
current_directory = dir;
$('#directory').val(dir);
window.location.hash = dir;
dust.render('explorer', base.push(d), function(err, out) {
$('#panel').html(out);
@ -169,7 +182,12 @@
var b = function() { browse_directory($('#directory').val()); };
$('#btn-nav-directory').click(b);
browse_directory('/');
var dir = window.location.hash.slice(1);
if(dir == "") {
window.location.hash = "/";
} else {
browse_directory(dir);
}
}
init();

View File

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

View File

@ -21,6 +21,7 @@
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
import java.util.UUID;
import java.util.regex.Pattern;
import org.apache.commons.lang.StringUtils;
@ -1175,4 +1176,32 @@ public void testReservedHdfsPathsOnFS() throws Exception {
public void testReservedHdfsPathsOnFC() throws Exception {
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.Random;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.hdfs.net.TcpPeerServer;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -38,6 +42,8 @@
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.net.NetUtils;
import org.apache.log4j.Level;
import org.apache.log4j.LogManager;
/**
* A helper class to setup the cluster, and get to BlockReader and DataNode for a block.
@ -141,22 +147,54 @@ public void readAndCheckEOS(BlockReader reader, int length, boolean expectEof)
*/
public BlockReader getBlockReader(LocatedBlock testBlock, int offset, int lenToRead)
throws IOException {
return getBlockReader(cluster, testBlock, offset, lenToRead);
}
/**
* Get a BlockReader for the given block.
*/
public static BlockReader getBlockReader(MiniDFSCluster cluster,
LocatedBlock testBlock, int offset, int lenToRead) throws IOException {
InetSocketAddress targetAddr = null;
Socket sock = null;
ExtendedBlock block = testBlock.getBlock();
DatanodeInfo[] nodes = testBlock.getLocations();
targetAddr = NetUtils.createSocketAddr(nodes[0].getXferAddr());
sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
sock.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT);
sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
return BlockReaderFactory.newBlockReader(
new DFSClient.Conf(conf),
targetAddr.toString()+ ":" + block.getBlockId(), block,
testBlock.getBlockToken(),
offset, lenToRead,
true, "BlockReaderTestUtil", TcpPeerServer.peerFromSocket(sock),
nodes[0], null, null, null, false, CachingStrategy.newDefaultStrategy());
final DistributedFileSystem fs = cluster.getFileSystem();
return new BlockReaderFactory(fs.getClient().getConf()).
setInetSocketAddress(targetAddr).
setBlock(block).
setFileName(targetAddr.toString()+ ":" + block.getBlockId()).
setBlockToken(testBlock.getBlockToken()).
setStartOffset(offset).
setLength(lenToRead).
setVerifyChecksum(true).
setClientName("BlockReaderTestUtil").
setDatanodeInfo(nodes[0]).
setClientCacheContext(ClientContext.getFromConf(fs.getConf())).
setCachingStrategy(CachingStrategy.newDefaultStrategy()).
setConfiguration(fs.getConf()).
setAllowShortCircuitLocalReads(true).
setRemotePeerFactory(new RemotePeerFactory() {
@Override
public Peer newConnectedPeer(InetSocketAddress addr)
throws IOException {
Peer peer = null;
Socket sock = NetUtils.
getDefaultSocketFactory(fs.getConf()).createSocket();
try {
sock.connect(addr, HdfsServerConstants.READ_TIMEOUT);
sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
peer = TcpPeerServer.peerFromSocket(sock);
} finally {
if (peer == null) {
IOUtils.closeQuietly(sock);
}
}
return peer;
}
}).
build();
}
/**
@ -167,4 +205,13 @@ public DataNode getDataNode(LocatedBlock testBlock) {
int ipcport = nodes[0].getIpcPort();
return cluster.getDataNode(ipcport);
}
}
public static void enableBlockReaderFactoryTracing() {
LogManager.getLogger(BlockReaderFactory.class.getName()).setLevel(
Level.TRACE);
LogManager.getLogger(ShortCircuitCache.class.getName()).setLevel(
Level.TRACE);
LogManager.getLogger(ShortCircuitReplica.class.getName()).setLevel(
Level.TRACE);
}
}

View File

@ -186,10 +186,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();
IOUtils.copyBytes(fs.open(fileName), os, 1024, true);
return os.toString();
try {
FSDataInputStream in = fs.open(fileName);
try {
IOUtils.copyBytes(fs.open(fileName), os, 1024, true);
return os.toByteArray();
} finally {
in.close();
}
} finally {
os.close();
}
}
public static void createFile(FileSystem fs, Path fileName, long fileLen,
@ -231,6 +247,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. */
public boolean checkFiles(FileSystem fs, String topdir) throws IOException {
Path root = new Path(topdir);
@ -550,8 +573,12 @@ public void cleanup(FileSystem fs, String topdir) throws IOException {
public static ExtendedBlock getFirstBlock(FileSystem fs, Path path) throws IOException {
HdfsDataInputStream in = (HdfsDataInputStream) fs.open(path);
in.readByte();
return in.getCurrentBlock();
try {
in.readByte();
return in.getCurrentBlock();
} finally {
in.close();
}
}
public static List<LocatedBlock> getAllBlocks(FSDataInputStream in)

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

View File

@ -25,18 +25,8 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.security.token.Token;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Matchers;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
/**
* This class tests the client connection caching in a single node
@ -48,30 +38,6 @@ public class TestConnCache {
static final int BLOCK_SIZE = 4096;
static final int FILE_SIZE = 3 * BLOCK_SIZE;
/**
* A mock Answer to remember the BlockReader used.
*
* It verifies that all invocation to DFSInputStream.getBlockReader()
* use the same peer.
*/
private class MockGetBlockReader implements Answer<RemoteBlockReader2> {
public RemoteBlockReader2 reader = null;
private Peer peer = null;
@Override
public RemoteBlockReader2 answer(InvocationOnMock invocation) throws Throwable {
RemoteBlockReader2 prevReader = reader;
reader = (RemoteBlockReader2) invocation.callRealMethod();
if (peer == null) {
peer = reader.getPeer();
} else if (prevReader != null) {
Assert.assertSame("DFSInputStream should use the same peer",
peer, reader.getPeer());
}
return reader;
}
}
/**
* (Optionally) seek to position, read and verify data.
*
@ -115,33 +81,29 @@ private void pread(DFSInputStream in,
* @throws Exception
*/
@Test
@SuppressWarnings("unchecked")
public void testReadFromOneDN() throws Exception {
BlockReaderTestUtil util = new BlockReaderTestUtil(1,
new HdfsConfiguration());
HdfsConfiguration configuration = new HdfsConfiguration();
// One of the goals of this test is to verify that we don't open more
// than one socket. So use a different client context, so that we
// get our own socket cache, rather than sharing with the other test
// instances. Also use a really long socket timeout so that nothing
// gets closed before we get around to checking the cache size at the end.
final String contextName = "testReadFromOneDNContext";
configuration.set(DFSConfigKeys.DFS_CLIENT_CONTEXT, contextName);
configuration.setLong(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY,
100000000L);
BlockReaderTestUtil util = new BlockReaderTestUtil(1, configuration);
final Path testFile = new Path("/testConnCache.dat");
byte authenticData[] = util.writeFile(testFile, FILE_SIZE / 1024);
DFSClient client = new DFSClient(
new InetSocketAddress("localhost",
util.getCluster().getNameNodePort()), util.getConf());
DFSInputStream in = Mockito.spy(client.open(testFile.toString()));
ClientContext cacheContext =
ClientContext.get(contextName, client.getConf());
DFSInputStream in = client.open(testFile.toString());
LOG.info("opened " + testFile.toString());
byte[] dataBuf = new byte[BLOCK_SIZE];
MockGetBlockReader answer = new MockGetBlockReader();
Mockito.doAnswer(answer).when(in).getBlockReader(
(InetSocketAddress) Matchers.anyObject(),
(DatanodeInfo) Matchers.anyObject(),
Matchers.anyString(),
(ExtendedBlock) Matchers.anyObject(),
(Token<BlockTokenIdentifier>) Matchers.anyObject(),
Matchers.anyLong(),
Matchers.anyLong(),
Matchers.anyInt(),
Matchers.anyBoolean(),
Matchers.anyString(),
(CachingStrategy)Matchers.anyObject());
// Initial read
pread(in, 0, dataBuf, 0, dataBuf.length, authenticData);
// Read again and verify that the socket is the same
@ -153,5 +115,8 @@ public void testReadFromOneDN() throws Exception {
pread(in, 64, dataBuf, 0, dataBuf.length / 2, authenticData);
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_WRITE_TIMEOUT_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
@ -86,21 +86,22 @@ public void testDatanodeRespectsKeepAliveTimeout() throws Exception {
// the datanode-side expiration time.
final long CLIENT_EXPIRY_MS = 60000L;
clientConf.setLong(DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY, CLIENT_EXPIRY_MS);
PeerCache.setInstance(DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT, CLIENT_EXPIRY_MS);
clientConf.set(DFS_CLIENT_CONTEXT, "testDatanodeRespectsKeepAliveTimeout");
DistributedFileSystem fs =
(DistributedFileSystem)FileSystem.get(cluster.getURI(),
clientConf);
PeerCache peerCache = ClientContext.getFromConf(clientConf).getPeerCache();
DFSTestUtil.createFile(fs, TEST_FILE, 1L, (short)1, 0L);
// Clients that write aren't currently re-used.
assertEquals(0, fs.dfs.peerCache.size());
assertEquals(0, peerCache.size());
assertXceiverCount(0);
// Reads the file, so we should get a
// cached socket, and should have an xceiver on the other side.
DFSTestUtil.readFile(fs, TEST_FILE);
assertEquals(1, fs.dfs.peerCache.size());
assertEquals(1, peerCache.size());
assertXceiverCount(1);
// Sleep for a bit longer than the keepalive timeout
@ -111,15 +112,13 @@ public void testDatanodeRespectsKeepAliveTimeout() throws Exception {
// The socket is still in the cache, because we don't
// notice that it's closed until we try to read
// from it again.
assertEquals(1, fs.dfs.peerCache.size());
assertEquals(1, peerCache.size());
// Take it out of the cache - reading should
// give an EOF.
Peer peer = fs.dfs.peerCache.get(dn.getDatanodeId(), false);
Peer peer = peerCache.get(dn.getDatanodeId(), false);
assertNotNull(peer);
assertEquals(-1, peer.getInputStream().read());
PeerCache.setInstance(DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT,
DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT);
}
/**
@ -132,34 +131,33 @@ public void testClientResponsesKeepAliveTimeout() throws Exception {
// the datanode-side expiration time.
final long CLIENT_EXPIRY_MS = 10L;
clientConf.setLong(DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY, CLIENT_EXPIRY_MS);
PeerCache.setInstance(DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT, CLIENT_EXPIRY_MS);
clientConf.set(DFS_CLIENT_CONTEXT, "testClientResponsesKeepAliveTimeout");
DistributedFileSystem fs =
(DistributedFileSystem)FileSystem.get(cluster.getURI(),
clientConf);
PeerCache peerCache = ClientContext.getFromConf(clientConf).getPeerCache();
DFSTestUtil.createFile(fs, TEST_FILE, 1L, (short)1, 0L);
// Clients that write aren't currently re-used.
assertEquals(0, fs.dfs.peerCache.size());
assertEquals(0, peerCache.size());
assertXceiverCount(0);
// Reads the file, so we should get a
// cached socket, and should have an xceiver on the other side.
DFSTestUtil.readFile(fs, TEST_FILE);
assertEquals(1, fs.dfs.peerCache.size());
assertEquals(1, peerCache.size());
assertXceiverCount(1);
// Sleep for a bit longer than the client keepalive timeout.
Thread.sleep(CLIENT_EXPIRY_MS + 1);
// Taking out a peer which is expired should give a null.
Peer peer = fs.dfs.peerCache.get(dn.getDatanodeId(), false);
Peer peer = peerCache.get(dn.getDatanodeId(), false);
assertTrue(peer == null);
// The socket cache is now empty.
assertEquals(0, fs.dfs.peerCache.size());
PeerCache.setInstance(DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT,
DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT);
assertEquals(0, peerCache.size());
}
/**
@ -174,7 +172,7 @@ public void testSlowReader() throws Exception {
final long CLIENT_EXPIRY_MS = 600000L;
Configuration clientConf = new Configuration(conf);
clientConf.setLong(DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY, CLIENT_EXPIRY_MS);
PeerCache.setInstance(DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT, CLIENT_EXPIRY_MS);
clientConf.set(DFS_CLIENT_CONTEXT, "testSlowReader");
DistributedFileSystem fs =
(DistributedFileSystem)FileSystem.get(cluster.getURI(),
clientConf);
@ -209,7 +207,12 @@ public Boolean get() {
@Test(timeout=30000)
public void testManyClosedSocketsInCache() throws Exception {
// Make a small file
DistributedFileSystem fs = cluster.getFileSystem();
Configuration clientConf = new Configuration(conf);
clientConf.set(DFS_CLIENT_CONTEXT, "testManyClosedSocketsInCache");
DistributedFileSystem fs =
(DistributedFileSystem)FileSystem.get(cluster.getURI(),
clientConf);
PeerCache peerCache = ClientContext.getFromConf(clientConf).getPeerCache();
DFSTestUtil.createFile(fs, TEST_FILE, 1L, (short)1, 0L);
// Insert a bunch of dead sockets in the cache, by opening
@ -227,15 +230,14 @@ public void testManyClosedSocketsInCache() throws Exception {
IOUtils.cleanup(null, stms);
}
DFSClient client = ((DistributedFileSystem)fs).dfs;
assertEquals(5, client.peerCache.size());
assertEquals(5, peerCache.size());
// Let all the xceivers timeout
Thread.sleep(1500);
assertXceiverCount(0);
// Client side still has the sockets cached
assertEquals(5, client.peerCache.size());
assertEquals(5, peerCache.size());
// Reading should not throw an exception.
DFSTestUtil.readFile(fs, TEST_FILE);

View File

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

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

View File

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

View File

@ -0,0 +1,161 @@
/**
* 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.server.blockmanagement;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.test.PathUtils;
import org.apache.hadoop.util.VersionInfo;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
public class TestReplicationPolicyConsiderLoad {
private static NameNode namenode;
private static DatanodeManager dnManager;
private static List<DatanodeRegistration> dnrList;
private static DatanodeDescriptor[] dataNodes;
private static DatanodeStorageInfo[] storages;
@BeforeClass
public static void setupCluster() throws IOException {
Configuration conf = new HdfsConfiguration();
final String[] racks = {
"/rack1",
"/rack1",
"/rack1",
"/rack2",
"/rack2",
"/rack2"};
storages = DFSTestUtil.createDatanodeStorageInfos(racks);
dataNodes = DFSTestUtil.toDatanodeDescriptor(storages);
FileSystem.setDefaultUri(conf, "hdfs://localhost:0");
conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
File baseDir = PathUtils.getTestDir(TestReplicationPolicy.class);
conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
new File(baseDir, "name").getPath());
conf.setBoolean(
DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY, true);
conf.setBoolean(
DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_KEY, true);
conf.setBoolean(
DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, true);
DFSTestUtil.formatNameNode(conf);
namenode = new NameNode(conf);
int blockSize = 1024;
dnrList = new ArrayList<DatanodeRegistration>();
dnManager = namenode.getNamesystem().getBlockManager().getDatanodeManager();
// Register DNs
for (int i=0; i < 6; i++) {
DatanodeRegistration dnr = new DatanodeRegistration(dataNodes[i],
new StorageInfo(), new ExportedBlockKeys(), VersionInfo.getVersion());
dnrList.add(dnr);
dnManager.registerDatanode(dnr);
dataNodes[i].getStorageInfos()[0].setUtilizationForTesting(
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*blockSize, 0L,
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*blockSize, 0L);
dataNodes[i].updateHeartbeat(
BlockManagerTestUtil.getStorageReportsForDatanode(dataNodes[i]),
0L, 0L, 0, 0);
}
}
/**
* Tests that chooseTarget with considerLoad set to true correctly calculates
* load with decommissioned nodes.
*/
@Test
public void testChooseTargetWithDecomNodes() throws IOException {
namenode.getNamesystem().writeLock();
try {
// Decommission DNs so BlockPlacementPolicyDefault.isGoodTarget()
// returns false
for (int i = 0; i < 3; i++) {
DatanodeInfo d = dnManager.getDatanodeByXferAddr(
dnrList.get(i).getIpAddr(),
dnrList.get(i).getXferPort());
d.setDecommissioned();
}
String blockPoolId = namenode.getNamesystem().getBlockPoolId();
dnManager.handleHeartbeat(dnrList.get(3),
BlockManagerTestUtil.getStorageReportsForDatanode(dataNodes[3]),
blockPoolId, dataNodes[3].getCacheCapacity(),
dataNodes[3].getCacheRemaining(),
2, 0, 0);
dnManager.handleHeartbeat(dnrList.get(4),
BlockManagerTestUtil.getStorageReportsForDatanode(dataNodes[4]),
blockPoolId, dataNodes[4].getCacheCapacity(),
dataNodes[4].getCacheRemaining(),
4, 0, 0);
dnManager.handleHeartbeat(dnrList.get(5),
BlockManagerTestUtil.getStorageReportsForDatanode(dataNodes[5]),
blockPoolId, dataNodes[5].getCacheCapacity(),
dataNodes[5].getCacheRemaining(),
4, 0, 0);
// Call chooseTarget()
DatanodeStorageInfo[] targets = namenode.getNamesystem().getBlockManager()
.getBlockPlacementPolicy().chooseTarget("testFile.txt", 3,
dataNodes[0], new ArrayList<DatanodeStorageInfo>(), false, null,
1024, StorageType.DEFAULT);
assertEquals(3, targets.length);
Set<DatanodeStorageInfo> targetSet = new HashSet<DatanodeStorageInfo>(
Arrays.asList(targets));
for (int i = 3; i < storages.length; i++) {
assertTrue(targetSet.contains(storages[i]));
}
} finally {
dataNodes[0].stopDecommission();
dataNodes[1].stopDecommission();
dataNodes[2].stopDecommission();
namenode.getNamesystem().writeUnlock();
}
NameNode.LOG.info("Done working on it");
}
@AfterClass
public static void teardownCluster() {
if (namenode != null) namenode.stop();
}
}

View File

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

View File

@ -0,0 +1,36 @@
/**
* 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.server.namenode;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext.DeduplicationMap;
import org.junit.Assert;
import org.junit.Test;
public class TestDeduplicationMap {
@Test
public void testDeduplicationMap() {
DeduplicationMap<String> m = DeduplicationMap.newMap();
Assert.assertEquals(1, m.getId("1"));
Assert.assertEquals(2, m.getId("2"));
Assert.assertEquals(3, m.getId("3"));
Assert.assertEquals(1, m.getId("1"));
Assert.assertEquals(2, m.getId("2"));
Assert.assertEquals(3, m.getId("3"));
}
}

View File

@ -0,0 +1,138 @@
/**
* 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.server.namenode;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.IOException;
import java.util.EnumSet;
import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSOutputStream;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
import org.junit.Test;
public class TestFSImage {
@Test
public void testPersist() throws IOException {
Configuration conf = new Configuration();
testPersistHelper(conf);
}
@Test
public void testCompression() throws IOException {
Configuration conf = new Configuration();
conf.setBoolean(DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, true);
conf.set(DFSConfigKeys.DFS_IMAGE_COMPRESSION_CODEC_KEY,
"org.apache.hadoop.io.compress.GzipCodec");
testPersistHelper(conf);
}
private void testPersistHelper(Configuration conf) throws IOException {
MiniDFSCluster cluster = null;
try {
cluster = new MiniDFSCluster.Builder(conf).build();
cluster.waitActive();
FSNamesystem fsn = cluster.getNamesystem();
DistributedFileSystem fs = cluster.getFileSystem();
final Path dir = new Path("/abc/def");
final Path file1 = new Path(dir, "f1");
final Path file2 = new Path(dir, "f2");
// create an empty file f1
fs.create(file1).close();
// create an under-construction file f2
FSDataOutputStream out = fs.create(file2);
out.writeBytes("hello");
((DFSOutputStream) out.getWrappedStream()).hsync(EnumSet
.of(SyncFlag.UPDATE_LENGTH));
// checkpoint
fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
fs.saveNamespace();
fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
cluster.restartNameNode();
cluster.waitActive();
fs = cluster.getFileSystem();
assertTrue(fs.isDirectory(dir));
assertTrue(fs.exists(file1));
assertTrue(fs.exists(file2));
// check internals of file2
INodeFile file2Node = fsn.dir.getINode4Write(file2.toString()).asFile();
assertEquals("hello".length(), file2Node.computeFileSize());
assertTrue(file2Node.isUnderConstruction());
BlockInfo[] blks = file2Node.getBlocks();
assertEquals(1, blks.length);
assertEquals(BlockUCState.UNDER_CONSTRUCTION, blks[0].getBlockUCState());
// check lease manager
Lease lease = fsn.leaseManager.getLeaseByPath(file2.toString());
Assert.assertNotNull(lease);
} finally {
if (cluster != null) {
cluster.shutdown();
}
}
}
/**
* Ensure that the digest written by the saver equals to the digest of the
* file.
*/
@Test
public void testDigest() throws IOException {
Configuration conf = new Configuration();
MiniDFSCluster cluster = null;
try {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
DistributedFileSystem fs = cluster.getFileSystem();
fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
fs.saveNamespace();
fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
File currentDir = FSImageTestUtil.getNameNodeCurrentDirs(cluster, 0).get(
0);
File fsimage = FSImageTestUtil.findNewestImageFile(currentDir
.getAbsolutePath());
assertEquals(MD5FileUtils.readStoredMd5ForFile(fsimage),
MD5FileUtils.computeMd5ForFile(fsimage));
} finally {
if (cluster != null) {
cluster.shutdown();
}
}
}
}

View File

@ -27,17 +27,12 @@
import java.io.File;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.junit.Test;
public class TestFSImageStorageInspector {
private static final Log LOG = LogFactory.getLog(
TestFSImageStorageInspector.class);
/**
* Simple test with image, edits, and inprogress edits
*/

Some files were not shown because too many files have changed in this diff Show More