HDFS-2197. Refactor RPC call implementations out of NameNode class. Contributed by Todd Lipcon.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1165463 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d1438b501d
commit
b0632df93a
@ -11,6 +11,7 @@ Trunk (unreleased changes)
|
|||||||
|
|
||||||
HDFS-1620. Rename HdfsConstants -> HdfsServerConstants, FSConstants ->
|
HDFS-1620. Rename HdfsConstants -> HdfsServerConstants, FSConstants ->
|
||||||
HdfsConstants. (Harsh J Chouraria via atm)
|
HdfsConstants. (Harsh J Chouraria via atm)
|
||||||
|
HDFS-2197. Refactor RPC call implementations out of NameNode class (todd)
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
HDFS-2287. TestParallelRead has a small off-by-one bug. (todd)
|
HDFS-2287. TestParallelRead has a small off-by-one bug. (todd)
|
||||||
|
@ -52,7 +52,7 @@
|
|||||||
* </ol>
|
* </ol>
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class BackupNode extends NameNode implements JournalProtocol {
|
public class BackupNode extends NameNode {
|
||||||
private static final String BN_ADDRESS_NAME_KEY = DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY;
|
private static final String BN_ADDRESS_NAME_KEY = DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY;
|
||||||
private static final String BN_ADDRESS_DEFAULT = DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_DEFAULT;
|
private static final String BN_ADDRESS_DEFAULT = DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_DEFAULT;
|
||||||
private static final String BN_HTTP_ADDRESS_NAME_KEY = DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY;
|
private static final String BN_HTTP_ADDRESS_NAME_KEY = DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY;
|
||||||
@ -74,7 +74,6 @@ public class BackupNode extends NameNode implements JournalProtocol {
|
|||||||
|
|
||||||
BackupNode(Configuration conf, NamenodeRole role) throws IOException {
|
BackupNode(Configuration conf, NamenodeRole role) throws IOException {
|
||||||
super(conf, role);
|
super(conf, role);
|
||||||
this.server.addProtocol(JournalProtocol.class, this);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/////////////////////////////////////////////////////
|
/////////////////////////////////////////////////////
|
||||||
@ -96,18 +95,20 @@ protected InetSocketAddress getServiceRpcServerAddress(Configuration conf) throw
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override // NameNode
|
@Override // NameNode
|
||||||
protected void setRpcServerAddress(Configuration conf) {
|
protected void setRpcServerAddress(Configuration conf,
|
||||||
conf.set(BN_ADDRESS_NAME_KEY, getHostPortString(rpcAddress));
|
InetSocketAddress addr) {
|
||||||
|
conf.set(BN_ADDRESS_NAME_KEY, getHostPortString(addr));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override // Namenode
|
@Override // Namenode
|
||||||
protected void setRpcServiceServerAddress(Configuration conf) {
|
protected void setRpcServiceServerAddress(Configuration conf,
|
||||||
conf.set(BN_SERVICE_RPC_ADDRESS_KEY, getHostPortString(serviceRPCAddress));
|
InetSocketAddress addr) {
|
||||||
|
conf.set(BN_SERVICE_RPC_ADDRESS_KEY, getHostPortString(addr));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override // NameNode
|
@Override // NameNode
|
||||||
protected InetSocketAddress getHttpServerAddress(Configuration conf) {
|
protected InetSocketAddress getHttpServerAddress(Configuration conf) {
|
||||||
assert rpcAddress != null : "rpcAddress should be calculated first";
|
assert getNameNodeAddress() != null : "rpcAddress should be calculated first";
|
||||||
String addr = conf.get(BN_HTTP_ADDRESS_NAME_KEY, BN_HTTP_ADDRESS_DEFAULT);
|
String addr = conf.get(BN_HTTP_ADDRESS_NAME_KEY, BN_HTTP_ADDRESS_DEFAULT);
|
||||||
return NetUtils.createSocketAddr(addr);
|
return NetUtils.createSocketAddr(addr);
|
||||||
}
|
}
|
||||||
@ -146,6 +147,12 @@ protected void initialize(Configuration conf) throws IOException {
|
|||||||
runCheckpointDaemon(conf);
|
runCheckpointDaemon(conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected NameNodeRpcServer createRpcServer(Configuration conf)
|
||||||
|
throws IOException {
|
||||||
|
return new BackupNodeRpcServer(conf, this);
|
||||||
|
}
|
||||||
|
|
||||||
@Override // NameNode
|
@Override // NameNode
|
||||||
public void stop() {
|
public void stop() {
|
||||||
if(checkpointManager != null) {
|
if(checkpointManager != null) {
|
||||||
@ -178,6 +185,15 @@ public void stop() {
|
|||||||
super.stop();
|
super.stop();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static class BackupNodeRpcServer extends NameNodeRpcServer implements JournalProtocol {
|
||||||
|
private final String nnRpcAddress;
|
||||||
|
|
||||||
|
private BackupNodeRpcServer(Configuration conf, BackupNode nn)
|
||||||
|
throws IOException {
|
||||||
|
super(conf, nn);
|
||||||
|
this.server.addProtocol(JournalProtocol.class, this);
|
||||||
|
nnRpcAddress = nn.nnRpcAddress;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getProtocolVersion(String protocol, long clientVersion)
|
public long getProtocolVersion(String protocol, long clientVersion)
|
||||||
@ -228,7 +244,7 @@ public void journal(NamenodeRegistration nnReg,
|
|||||||
verifyRequest(nnReg);
|
verifyRequest(nnReg);
|
||||||
if(!nnRpcAddress.equals(nnReg.getAddress()))
|
if(!nnRpcAddress.equals(nnReg.getAddress()))
|
||||||
throw new IOException("Journal request from unexpected name-node: "
|
throw new IOException("Journal request from unexpected name-node: "
|
||||||
+ nnReg.getAddress() + " expecting " + nnRpcAddress);
|
+ nnReg.getAddress() + " expecting " + rpcAddress);
|
||||||
getBNImage().journal(firstTxId, numTxns, records);
|
getBNImage().journal(firstTxId, numTxns, records);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -240,13 +256,14 @@ public void startLogSegment(NamenodeRegistration registration, long txid)
|
|||||||
getBNImage().namenodeStartedLogSegment(txid);
|
getBNImage().namenodeStartedLogSegment(txid);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private BackupImage getBNImage() {
|
||||||
|
return (BackupImage)nn.getFSImage();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
//////////////////////////////////////////////////////
|
//////////////////////////////////////////////////////
|
||||||
|
|
||||||
|
|
||||||
BackupImage getBNImage() {
|
|
||||||
return (BackupImage)getFSImage();
|
|
||||||
}
|
|
||||||
|
|
||||||
boolean shouldCheckpointAtStartup() {
|
boolean shouldCheckpointAtStartup() {
|
||||||
FSImage fsImage = getFSImage();
|
FSImage fsImage = getFSImage();
|
||||||
if(isRole(NamenodeRole.CHECKPOINT)) {
|
if(isRole(NamenodeRole.CHECKPOINT)) {
|
||||||
|
@ -69,7 +69,7 @@ protected void doGet(final HttpServletRequest req, final HttpServletResponse res
|
|||||||
try {
|
try {
|
||||||
ugi.doAs(new PrivilegedExceptionAction<Void>() {
|
ugi.doAs(new PrivilegedExceptionAction<Void>() {
|
||||||
public Void run() throws Exception {
|
public Void run() throws Exception {
|
||||||
nn.cancelDelegationToken(token);
|
nn.getRpcServer().cancelDelegationToken(token);
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -73,7 +73,7 @@ protected ClientProtocol createNameNodeProxy() throws IOException {
|
|||||||
// rpc
|
// rpc
|
||||||
NameNode nn = NameNodeHttpServer.getNameNodeFromContext(context);
|
NameNode nn = NameNodeHttpServer.getNameNodeFromContext(context);
|
||||||
if (nn != null) {
|
if (nn != null) {
|
||||||
return nn;
|
return nn.getRpcServer();
|
||||||
}
|
}
|
||||||
InetSocketAddress nnAddr =
|
InetSocketAddress nnAddr =
|
||||||
NameNodeHttpServer.getNameNodeAddressFromContext(context);
|
NameNodeHttpServer.getNameNodeAddressFromContext(context);
|
||||||
|
@ -75,7 +75,7 @@ public Void run() throws Exception {
|
|||||||
+ ":" + NameNode.getAddress(conf).getPort();
|
+ ":" + NameNode.getAddress(conf).getPort();
|
||||||
|
|
||||||
Token<DelegationTokenIdentifier> token =
|
Token<DelegationTokenIdentifier> token =
|
||||||
nn.getDelegationToken(new Text(renewerFinal));
|
nn.getRpcServer().getDelegationToken(new Text(renewerFinal));
|
||||||
if(token == null) {
|
if(token == null) {
|
||||||
throw new Exception("couldn't get the token for " +s);
|
throw new Exception("couldn't get the token for " +s);
|
||||||
}
|
}
|
||||||
|
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,918 @@
|
|||||||
|
/**
|
||||||
|
* 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.apache.hadoop.hdfs.protocol.HdfsConstants.MAX_PATH_DEPTH;
|
||||||
|
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.MAX_PATH_LENGTH;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.net.InetSocketAddress;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.HashMap;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||||
|
import org.apache.hadoop.fs.ContentSummary;
|
||||||
|
import org.apache.hadoop.fs.CreateFlag;
|
||||||
|
import org.apache.hadoop.fs.FsServerDefaults;
|
||||||
|
import org.apache.hadoop.fs.Options;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||||
|
import org.apache.hadoop.fs.permission.FsPermission;
|
||||||
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
||||||
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
|
||||||
|
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
|
||||||
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
|
||||||
|
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
|
||||||
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||||
|
import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
|
||||||
|
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
|
||||||
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
|
||||||
|
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.NodeRegistration;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
|
||||||
|
import org.apache.hadoop.io.EnumSetWritable;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.ipc.ProtocolSignature;
|
||||||
|
import org.apache.hadoop.ipc.RPC;
|
||||||
|
import org.apache.hadoop.ipc.Server;
|
||||||
|
import org.apache.hadoop.net.Node;
|
||||||
|
import org.apache.hadoop.security.AccessControlException;
|
||||||
|
import org.apache.hadoop.security.Groups;
|
||||||
|
import org.apache.hadoop.security.RefreshUserMappingsProtocol;
|
||||||
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
|
import org.apache.hadoop.security.authorize.AuthorizationException;
|
||||||
|
import org.apache.hadoop.security.authorize.ProxyUsers;
|
||||||
|
import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
|
||||||
|
import org.apache.hadoop.security.token.Token;
|
||||||
|
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||||
|
import org.apache.hadoop.tools.GetUserMappingsProtocol;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class is responsible for handling all of the RPC calls to the NameNode.
|
||||||
|
* It is created, started, and stopped by {@link NameNode}.
|
||||||
|
*/
|
||||||
|
class NameNodeRpcServer implements NamenodeProtocols {
|
||||||
|
|
||||||
|
private static final Log LOG = NameNode.LOG;
|
||||||
|
private static final Log stateChangeLog = NameNode.stateChangeLog;
|
||||||
|
|
||||||
|
// Dependencies from other parts of NN.
|
||||||
|
private final FSNamesystem namesystem;
|
||||||
|
protected final NameNode nn;
|
||||||
|
private final NameNodeMetrics metrics;
|
||||||
|
|
||||||
|
private final boolean serviceAuthEnabled;
|
||||||
|
|
||||||
|
/** The RPC server that listens to requests from DataNodes */
|
||||||
|
private final RPC.Server serviceRpcServer;
|
||||||
|
private final InetSocketAddress serviceRPCAddress;
|
||||||
|
|
||||||
|
/** The RPC server that listens to requests from clients */
|
||||||
|
protected final RPC.Server server;
|
||||||
|
protected final InetSocketAddress rpcAddress;
|
||||||
|
|
||||||
|
public NameNodeRpcServer(Configuration conf, NameNode nn)
|
||||||
|
throws IOException {
|
||||||
|
this.nn = nn;
|
||||||
|
this.namesystem = nn.getNamesystem();
|
||||||
|
this.metrics = NameNode.getNameNodeMetrics();
|
||||||
|
|
||||||
|
int handlerCount =
|
||||||
|
conf.getInt(DFS_DATANODE_HANDLER_COUNT_KEY,
|
||||||
|
DFS_DATANODE_HANDLER_COUNT_DEFAULT);
|
||||||
|
InetSocketAddress socAddr = nn.getRpcServerAddress(conf);
|
||||||
|
|
||||||
|
InetSocketAddress dnSocketAddr = nn.getServiceRpcServerAddress(conf);
|
||||||
|
if (dnSocketAddr != null) {
|
||||||
|
int serviceHandlerCount =
|
||||||
|
conf.getInt(DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY,
|
||||||
|
DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT);
|
||||||
|
this.serviceRpcServer = RPC.getServer(NamenodeProtocols.class, this,
|
||||||
|
dnSocketAddr.getHostName(), dnSocketAddr.getPort(), serviceHandlerCount,
|
||||||
|
false, conf, namesystem.getDelegationTokenSecretManager());
|
||||||
|
this.serviceRPCAddress = this.serviceRpcServer.getListenerAddress();
|
||||||
|
nn.setRpcServiceServerAddress(conf, serviceRPCAddress);
|
||||||
|
} else {
|
||||||
|
serviceRpcServer = null;
|
||||||
|
serviceRPCAddress = null;
|
||||||
|
}
|
||||||
|
this.server = RPC.getServer(NamenodeProtocols.class, this,
|
||||||
|
socAddr.getHostName(), socAddr.getPort(),
|
||||||
|
handlerCount, false, conf,
|
||||||
|
namesystem.getDelegationTokenSecretManager());
|
||||||
|
|
||||||
|
// set service-level authorization security policy
|
||||||
|
if (serviceAuthEnabled =
|
||||||
|
conf.getBoolean(
|
||||||
|
CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
|
||||||
|
this.server.refreshServiceAcl(conf, new HDFSPolicyProvider());
|
||||||
|
if (this.serviceRpcServer != null) {
|
||||||
|
this.serviceRpcServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// The rpc-server port can be ephemeral... ensure we have the correct info
|
||||||
|
this.rpcAddress = this.server.getListenerAddress();
|
||||||
|
nn.setRpcServerAddress(conf, rpcAddress);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Actually start serving requests.
|
||||||
|
*/
|
||||||
|
void start() {
|
||||||
|
server.start(); //start RPC server
|
||||||
|
if (serviceRpcServer != null) {
|
||||||
|
serviceRpcServer.start();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Wait until the RPC server has shut down.
|
||||||
|
*/
|
||||||
|
void join() throws InterruptedException {
|
||||||
|
this.server.join();
|
||||||
|
}
|
||||||
|
|
||||||
|
void stop() {
|
||||||
|
if(server != null) server.stop();
|
||||||
|
if(serviceRpcServer != null) serviceRpcServer.stop();
|
||||||
|
}
|
||||||
|
|
||||||
|
InetSocketAddress getServiceRpcAddress() {
|
||||||
|
return serviceRPCAddress;
|
||||||
|
}
|
||||||
|
|
||||||
|
InetSocketAddress getRpcAddress() {
|
||||||
|
return rpcAddress;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // VersionedProtocol
|
||||||
|
public ProtocolSignature getProtocolSignature(String protocol,
|
||||||
|
long clientVersion, int clientMethodsHash) throws IOException {
|
||||||
|
return ProtocolSignature.getProtocolSignature(
|
||||||
|
this, protocol, clientVersion, clientMethodsHash);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getProtocolVersion(String protocol,
|
||||||
|
long clientVersion) throws IOException {
|
||||||
|
if (protocol.equals(ClientProtocol.class.getName())) {
|
||||||
|
return ClientProtocol.versionID;
|
||||||
|
} else if (protocol.equals(DatanodeProtocol.class.getName())){
|
||||||
|
return DatanodeProtocol.versionID;
|
||||||
|
} else if (protocol.equals(NamenodeProtocol.class.getName())){
|
||||||
|
return NamenodeProtocol.versionID;
|
||||||
|
} else if (protocol.equals(RefreshAuthorizationPolicyProtocol.class.getName())){
|
||||||
|
return RefreshAuthorizationPolicyProtocol.versionID;
|
||||||
|
} else if (protocol.equals(RefreshUserMappingsProtocol.class.getName())){
|
||||||
|
return RefreshUserMappingsProtocol.versionID;
|
||||||
|
} else if (protocol.equals(GetUserMappingsProtocol.class.getName())){
|
||||||
|
return GetUserMappingsProtocol.versionID;
|
||||||
|
} else {
|
||||||
|
throw new IOException("Unknown protocol to name node: " + protocol);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/////////////////////////////////////////////////////
|
||||||
|
// NamenodeProtocol
|
||||||
|
/////////////////////////////////////////////////////
|
||||||
|
@Override // NamenodeProtocol
|
||||||
|
public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
|
||||||
|
throws IOException {
|
||||||
|
if(size <= 0) {
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"Unexpected not positive size: "+size);
|
||||||
|
}
|
||||||
|
|
||||||
|
return namesystem.getBlockManager().getBlocks(datanode, size);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // NamenodeProtocol
|
||||||
|
public ExportedBlockKeys getBlockKeys() throws IOException {
|
||||||
|
return namesystem.getBlockManager().getBlockKeys();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // NamenodeProtocol
|
||||||
|
public void errorReport(NamenodeRegistration registration,
|
||||||
|
int errorCode,
|
||||||
|
String msg) throws IOException {
|
||||||
|
verifyRequest(registration);
|
||||||
|
LOG.info("Error report from " + registration + ": " + msg);
|
||||||
|
if(errorCode == FATAL)
|
||||||
|
namesystem.releaseBackupNode(registration);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // NamenodeProtocol
|
||||||
|
public NamenodeRegistration register(NamenodeRegistration registration)
|
||||||
|
throws IOException {
|
||||||
|
verifyVersion(registration.getVersion());
|
||||||
|
NamenodeRegistration myRegistration = nn.setRegistration();
|
||||||
|
namesystem.registerBackupNode(registration, myRegistration);
|
||||||
|
return myRegistration;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // NamenodeProtocol
|
||||||
|
public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
|
||||||
|
throws IOException {
|
||||||
|
verifyRequest(registration);
|
||||||
|
if(!nn.isRole(NamenodeRole.NAMENODE))
|
||||||
|
throw new IOException("Only an ACTIVE node can invoke startCheckpoint.");
|
||||||
|
return namesystem.startCheckpoint(registration, nn.setRegistration());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // NamenodeProtocol
|
||||||
|
public void endCheckpoint(NamenodeRegistration registration,
|
||||||
|
CheckpointSignature sig) throws IOException {
|
||||||
|
verifyRequest(registration);
|
||||||
|
if(!nn.isRole(NamenodeRole.NAMENODE))
|
||||||
|
throw new IOException("Only an ACTIVE node can invoke endCheckpoint.");
|
||||||
|
namesystem.endCheckpoint(registration, sig);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
|
||||||
|
throws IOException {
|
||||||
|
return namesystem.getDelegationToken(renewer);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
|
||||||
|
throws InvalidToken, IOException {
|
||||||
|
return namesystem.renewDelegationToken(token);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
|
||||||
|
throws IOException {
|
||||||
|
namesystem.cancelDelegationToken(token);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public LocatedBlocks getBlockLocations(String src,
|
||||||
|
long offset,
|
||||||
|
long length)
|
||||||
|
throws IOException {
|
||||||
|
metrics.incrGetBlockLocations();
|
||||||
|
return namesystem.getBlockLocations(getClientMachine(),
|
||||||
|
src, offset, length);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public FsServerDefaults getServerDefaults() throws IOException {
|
||||||
|
return namesystem.getServerDefaults();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void create(String src,
|
||||||
|
FsPermission masked,
|
||||||
|
String clientName,
|
||||||
|
EnumSetWritable<CreateFlag> flag,
|
||||||
|
boolean createParent,
|
||||||
|
short replication,
|
||||||
|
long blockSize) throws IOException {
|
||||||
|
String clientMachine = getClientMachine();
|
||||||
|
if (stateChangeLog.isDebugEnabled()) {
|
||||||
|
stateChangeLog.debug("*DIR* NameNode.create: file "
|
||||||
|
+src+" for "+clientName+" at "+clientMachine);
|
||||||
|
}
|
||||||
|
if (!checkPathLength(src)) {
|
||||||
|
throw new IOException("create: Pathname too long. Limit "
|
||||||
|
+ MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
|
||||||
|
}
|
||||||
|
namesystem.startFile(src,
|
||||||
|
new PermissionStatus(UserGroupInformation.getCurrentUser().getShortUserName(),
|
||||||
|
null, masked),
|
||||||
|
clientName, clientMachine, flag.get(), createParent, replication, blockSize);
|
||||||
|
metrics.incrFilesCreated();
|
||||||
|
metrics.incrCreateFileOps();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public LocatedBlock append(String src, String clientName)
|
||||||
|
throws IOException {
|
||||||
|
String clientMachine = getClientMachine();
|
||||||
|
if (stateChangeLog.isDebugEnabled()) {
|
||||||
|
stateChangeLog.debug("*DIR* NameNode.append: file "
|
||||||
|
+src+" for "+clientName+" at "+clientMachine);
|
||||||
|
}
|
||||||
|
LocatedBlock info = namesystem.appendFile(src, clientName, clientMachine);
|
||||||
|
metrics.incrFilesAppended();
|
||||||
|
return info;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public boolean recoverLease(String src, String clientName) throws IOException {
|
||||||
|
String clientMachine = getClientMachine();
|
||||||
|
return namesystem.recoverLease(src, clientName, clientMachine);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public boolean setReplication(String src, short replication)
|
||||||
|
throws IOException {
|
||||||
|
return namesystem.setReplication(src, replication);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void setPermission(String src, FsPermission permissions)
|
||||||
|
throws IOException {
|
||||||
|
namesystem.setPermission(src, permissions);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void setOwner(String src, String username, String groupname)
|
||||||
|
throws IOException {
|
||||||
|
namesystem.setOwner(src, username, groupname);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public LocatedBlock addBlock(String src,
|
||||||
|
String clientName,
|
||||||
|
ExtendedBlock previous,
|
||||||
|
DatanodeInfo[] excludedNodes)
|
||||||
|
throws IOException {
|
||||||
|
if(stateChangeLog.isDebugEnabled()) {
|
||||||
|
stateChangeLog.debug("*BLOCK* NameNode.addBlock: file "
|
||||||
|
+src+" for "+clientName);
|
||||||
|
}
|
||||||
|
HashMap<Node, Node> excludedNodesSet = null;
|
||||||
|
if (excludedNodes != null) {
|
||||||
|
excludedNodesSet = new HashMap<Node, Node>(excludedNodes.length);
|
||||||
|
for (Node node:excludedNodes) {
|
||||||
|
excludedNodesSet.put(node, node);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
LocatedBlock locatedBlock =
|
||||||
|
namesystem.getAdditionalBlock(src, clientName, previous, excludedNodesSet);
|
||||||
|
if (locatedBlock != null)
|
||||||
|
metrics.incrAddBlockOps();
|
||||||
|
return locatedBlock;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk,
|
||||||
|
final DatanodeInfo[] existings, final DatanodeInfo[] excludes,
|
||||||
|
final int numAdditionalNodes, final String clientName
|
||||||
|
) throws IOException {
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("getAdditionalDatanode: src=" + src
|
||||||
|
+ ", blk=" + blk
|
||||||
|
+ ", existings=" + Arrays.asList(existings)
|
||||||
|
+ ", excludes=" + Arrays.asList(excludes)
|
||||||
|
+ ", numAdditionalNodes=" + numAdditionalNodes
|
||||||
|
+ ", clientName=" + clientName);
|
||||||
|
}
|
||||||
|
|
||||||
|
metrics.incrGetAdditionalDatanodeOps();
|
||||||
|
|
||||||
|
HashMap<Node, Node> excludeSet = null;
|
||||||
|
if (excludes != null) {
|
||||||
|
excludeSet = new HashMap<Node, Node>(excludes.length);
|
||||||
|
for (Node node : excludes) {
|
||||||
|
excludeSet.put(node, node);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return namesystem.getAdditionalDatanode(src, blk,
|
||||||
|
existings, excludeSet, numAdditionalNodes, clientName);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The client needs to give up on the block.
|
||||||
|
*/
|
||||||
|
public void abandonBlock(ExtendedBlock b, String src, String holder)
|
||||||
|
throws IOException {
|
||||||
|
if(stateChangeLog.isDebugEnabled()) {
|
||||||
|
stateChangeLog.debug("*BLOCK* NameNode.abandonBlock: "
|
||||||
|
+b+" of file "+src);
|
||||||
|
}
|
||||||
|
if (!namesystem.abandonBlock(b, src, holder)) {
|
||||||
|
throw new IOException("Cannot abandon block during write to " + src);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public boolean complete(String src, String clientName, ExtendedBlock last)
|
||||||
|
throws IOException {
|
||||||
|
if(stateChangeLog.isDebugEnabled()) {
|
||||||
|
stateChangeLog.debug("*DIR* NameNode.complete: "
|
||||||
|
+ src + " for " + clientName);
|
||||||
|
}
|
||||||
|
return namesystem.completeFile(src, clientName, last);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The client has detected an error on the specified located blocks
|
||||||
|
* and is reporting them to the server. For now, the namenode will
|
||||||
|
* mark the block as corrupt. In the future we might
|
||||||
|
* check the blocks are actually corrupt.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
|
||||||
|
stateChangeLog.info("*DIR* NameNode.reportBadBlocks");
|
||||||
|
for (int i = 0; i < blocks.length; i++) {
|
||||||
|
ExtendedBlock blk = blocks[i].getBlock();
|
||||||
|
DatanodeInfo[] nodes = blocks[i].getLocations();
|
||||||
|
for (int j = 0; j < nodes.length; j++) {
|
||||||
|
DatanodeInfo dn = nodes[j];
|
||||||
|
namesystem.getBlockManager().findAndMarkBlockAsCorrupt(blk, dn);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public LocatedBlock updateBlockForPipeline(ExtendedBlock block, String clientName)
|
||||||
|
throws IOException {
|
||||||
|
return namesystem.updateBlockForPipeline(block, clientName);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void updatePipeline(String clientName, ExtendedBlock oldBlock,
|
||||||
|
ExtendedBlock newBlock, DatanodeID[] newNodes)
|
||||||
|
throws IOException {
|
||||||
|
namesystem.updatePipeline(clientName, oldBlock, newBlock, newNodes);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // DatanodeProtocol
|
||||||
|
public void commitBlockSynchronization(ExtendedBlock block,
|
||||||
|
long newgenerationstamp, long newlength,
|
||||||
|
boolean closeFile, boolean deleteblock, DatanodeID[] newtargets)
|
||||||
|
throws IOException {
|
||||||
|
namesystem.commitBlockSynchronization(block,
|
||||||
|
newgenerationstamp, newlength, closeFile, deleteblock, newtargets);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public long getPreferredBlockSize(String filename)
|
||||||
|
throws IOException {
|
||||||
|
return namesystem.getPreferredBlockSize(filename);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Deprecated
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public boolean rename(String src, String dst) throws IOException {
|
||||||
|
if(stateChangeLog.isDebugEnabled()) {
|
||||||
|
stateChangeLog.debug("*DIR* NameNode.rename: " + src + " to " + dst);
|
||||||
|
}
|
||||||
|
if (!checkPathLength(dst)) {
|
||||||
|
throw new IOException("rename: Pathname too long. Limit "
|
||||||
|
+ MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
|
||||||
|
}
|
||||||
|
boolean ret = namesystem.renameTo(src, dst);
|
||||||
|
if (ret) {
|
||||||
|
metrics.incrFilesRenamed();
|
||||||
|
}
|
||||||
|
return ret;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void concat(String trg, String[] src) throws IOException {
|
||||||
|
namesystem.concat(trg, src);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void rename(String src, String dst, Options.Rename... options)
|
||||||
|
throws IOException {
|
||||||
|
if(stateChangeLog.isDebugEnabled()) {
|
||||||
|
stateChangeLog.debug("*DIR* NameNode.rename: " + src + " to " + dst);
|
||||||
|
}
|
||||||
|
if (!checkPathLength(dst)) {
|
||||||
|
throw new IOException("rename: Pathname too long. Limit "
|
||||||
|
+ MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
|
||||||
|
}
|
||||||
|
namesystem.renameTo(src, dst, options);
|
||||||
|
metrics.incrFilesRenamed();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Deprecated
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public boolean delete(String src) throws IOException {
|
||||||
|
return delete(src, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public boolean delete(String src, boolean recursive) throws IOException {
|
||||||
|
if (stateChangeLog.isDebugEnabled()) {
|
||||||
|
stateChangeLog.debug("*DIR* Namenode.delete: src=" + src
|
||||||
|
+ ", recursive=" + recursive);
|
||||||
|
}
|
||||||
|
boolean ret = namesystem.delete(src, recursive);
|
||||||
|
if (ret)
|
||||||
|
metrics.incrDeleteFileOps();
|
||||||
|
return ret;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check path length does not exceed maximum. Returns true if
|
||||||
|
* length and depth are okay. Returns false if length is too long
|
||||||
|
* or depth is too great.
|
||||||
|
*/
|
||||||
|
private boolean checkPathLength(String src) {
|
||||||
|
Path srcPath = new Path(src);
|
||||||
|
return (src.length() <= MAX_PATH_LENGTH &&
|
||||||
|
srcPath.depth() <= MAX_PATH_DEPTH);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public boolean mkdirs(String src, FsPermission masked, boolean createParent)
|
||||||
|
throws IOException {
|
||||||
|
if(stateChangeLog.isDebugEnabled()) {
|
||||||
|
stateChangeLog.debug("*DIR* NameNode.mkdirs: " + src);
|
||||||
|
}
|
||||||
|
if (!checkPathLength(src)) {
|
||||||
|
throw new IOException("mkdirs: Pathname too long. Limit "
|
||||||
|
+ MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
|
||||||
|
}
|
||||||
|
return namesystem.mkdirs(src,
|
||||||
|
new PermissionStatus(UserGroupInformation.getCurrentUser().getShortUserName(),
|
||||||
|
null, masked), createParent);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void renewLease(String clientName) throws IOException {
|
||||||
|
namesystem.renewLease(clientName);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public DirectoryListing getListing(String src, byte[] startAfter,
|
||||||
|
boolean needLocation)
|
||||||
|
throws IOException {
|
||||||
|
DirectoryListing files = namesystem.getListing(
|
||||||
|
src, startAfter, needLocation);
|
||||||
|
if (files != null) {
|
||||||
|
metrics.incrGetListingOps();
|
||||||
|
metrics.incrFilesInGetListingOps(files.getPartialListing().length);
|
||||||
|
}
|
||||||
|
return files;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public HdfsFileStatus getFileInfo(String src) throws IOException {
|
||||||
|
metrics.incrFileInfoOps();
|
||||||
|
return namesystem.getFileInfo(src, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
|
||||||
|
metrics.incrFileInfoOps();
|
||||||
|
return namesystem.getFileInfo(src, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long[] getStats() {
|
||||||
|
return namesystem.getStats();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
|
||||||
|
throws IOException {
|
||||||
|
DatanodeInfo results[] = namesystem.datanodeReport(type);
|
||||||
|
if (results == null ) {
|
||||||
|
throw new IOException("Cannot find datanode report");
|
||||||
|
}
|
||||||
|
return results;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public boolean setSafeMode(SafeModeAction action) throws IOException {
|
||||||
|
return namesystem.setSafeMode(action);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public boolean restoreFailedStorage(String arg)
|
||||||
|
throws AccessControlException {
|
||||||
|
return namesystem.restoreFailedStorage(arg);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void saveNamespace() throws IOException {
|
||||||
|
namesystem.saveNamespace();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void refreshNodes() throws IOException {
|
||||||
|
namesystem.getBlockManager().getDatanodeManager().refreshNodes(
|
||||||
|
new HdfsConfiguration());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // NamenodeProtocol
|
||||||
|
public long getTransactionID() {
|
||||||
|
return namesystem.getEditLog().getSyncTxId();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // NamenodeProtocol
|
||||||
|
public CheckpointSignature rollEditLog() throws IOException {
|
||||||
|
return namesystem.rollEditLog();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
|
||||||
|
throws IOException {
|
||||||
|
return namesystem.getEditLog().getEditLogManifest(sinceTxId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void finalizeUpgrade() throws IOException {
|
||||||
|
namesystem.finalizeUpgrade();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
|
||||||
|
throws IOException {
|
||||||
|
return namesystem.distributedUpgradeProgress(action);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void metaSave(String filename) throws IOException {
|
||||||
|
namesystem.metaSave(filename);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
|
||||||
|
throws IOException {
|
||||||
|
Collection<FSNamesystem.CorruptFileBlockInfo> fbs =
|
||||||
|
namesystem.listCorruptFileBlocks(path, cookie);
|
||||||
|
|
||||||
|
String[] files = new String[fbs.size()];
|
||||||
|
String lastCookie = "";
|
||||||
|
int i = 0;
|
||||||
|
for(FSNamesystem.CorruptFileBlockInfo fb: fbs) {
|
||||||
|
files[i++] = fb.path;
|
||||||
|
lastCookie = fb.block.getBlockName();
|
||||||
|
}
|
||||||
|
return new CorruptFileBlocks(files, lastCookie);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tell all datanodes to use a new, non-persistent bandwidth value for
|
||||||
|
* dfs.datanode.balance.bandwidthPerSec.
|
||||||
|
* @param bandwidth Blanacer bandwidth in bytes per second for all datanodes.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void setBalancerBandwidth(long bandwidth) throws IOException {
|
||||||
|
namesystem.getBlockManager().getDatanodeManager().setBalancerBandwidth(bandwidth);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public ContentSummary getContentSummary(String path) throws IOException {
|
||||||
|
return namesystem.getContentSummary(path);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
|
||||||
|
throws IOException {
|
||||||
|
namesystem.setQuota(path, namespaceQuota, diskspaceQuota);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void fsync(String src, String clientName) throws IOException {
|
||||||
|
namesystem.fsync(src, clientName);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void setTimes(String src, long mtime, long atime)
|
||||||
|
throws IOException {
|
||||||
|
namesystem.setTimes(src, mtime, atime);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public void createSymlink(String target, String link, FsPermission dirPerms,
|
||||||
|
boolean createParent) throws IOException {
|
||||||
|
metrics.incrCreateSymlinkOps();
|
||||||
|
/* We enforce the MAX_PATH_LENGTH limit even though a symlink target
|
||||||
|
* URI may refer to a non-HDFS file system.
|
||||||
|
*/
|
||||||
|
if (!checkPathLength(link)) {
|
||||||
|
throw new IOException("Symlink path exceeds " + MAX_PATH_LENGTH +
|
||||||
|
" character limit");
|
||||||
|
|
||||||
|
}
|
||||||
|
if ("".equals(target)) {
|
||||||
|
throw new IOException("Invalid symlink target");
|
||||||
|
}
|
||||||
|
final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
|
||||||
|
namesystem.createSymlink(target, link,
|
||||||
|
new PermissionStatus(ugi.getShortUserName(), null, dirPerms), createParent);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // ClientProtocol
|
||||||
|
public String getLinkTarget(String path) throws IOException {
|
||||||
|
metrics.incrGetLinkTargetOps();
|
||||||
|
/* Resolves the first symlink in the given path, returning a
|
||||||
|
* new path consisting of the target of the symlink and any
|
||||||
|
* remaining path components from the original path.
|
||||||
|
*/
|
||||||
|
try {
|
||||||
|
HdfsFileStatus stat = namesystem.getFileInfo(path, false);
|
||||||
|
if (stat != null) {
|
||||||
|
// NB: getSymlink throws IOException if !stat.isSymlink()
|
||||||
|
return stat.getSymlink();
|
||||||
|
}
|
||||||
|
} catch (UnresolvedPathException e) {
|
||||||
|
return e.getResolvedPath().toString();
|
||||||
|
} catch (UnresolvedLinkException e) {
|
||||||
|
// The NameNode should only throw an UnresolvedPathException
|
||||||
|
throw new AssertionError("UnresolvedLinkException thrown");
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Override // DatanodeProtocol
|
||||||
|
public DatanodeRegistration registerDatanode(DatanodeRegistration nodeReg)
|
||||||
|
throws IOException {
|
||||||
|
verifyVersion(nodeReg.getVersion());
|
||||||
|
namesystem.registerDatanode(nodeReg);
|
||||||
|
|
||||||
|
return nodeReg;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // DatanodeProtocol
|
||||||
|
public DatanodeCommand[] sendHeartbeat(DatanodeRegistration nodeReg,
|
||||||
|
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
|
||||||
|
int xmitsInProgress, int xceiverCount, int failedVolumes)
|
||||||
|
throws IOException {
|
||||||
|
verifyRequest(nodeReg);
|
||||||
|
return namesystem.handleHeartbeat(nodeReg, capacity, dfsUsed, remaining,
|
||||||
|
blockPoolUsed, xceiverCount, xmitsInProgress, failedVolumes);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // DatanodeProtocol
|
||||||
|
public DatanodeCommand blockReport(DatanodeRegistration nodeReg,
|
||||||
|
String poolId, long[] blocks) throws IOException {
|
||||||
|
verifyRequest(nodeReg);
|
||||||
|
BlockListAsLongs blist = new BlockListAsLongs(blocks);
|
||||||
|
if(stateChangeLog.isDebugEnabled()) {
|
||||||
|
stateChangeLog.debug("*BLOCK* NameNode.blockReport: "
|
||||||
|
+ "from " + nodeReg.getName() + " " + blist.getNumberOfBlocks()
|
||||||
|
+ " blocks");
|
||||||
|
}
|
||||||
|
|
||||||
|
namesystem.getBlockManager().processReport(nodeReg, poolId, blist);
|
||||||
|
if (nn.getFSImage().isUpgradeFinalized())
|
||||||
|
return new DatanodeCommand.Finalize(poolId);
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // DatanodeProtocol
|
||||||
|
public void blockReceivedAndDeleted(DatanodeRegistration nodeReg, String poolId,
|
||||||
|
ReceivedDeletedBlockInfo[] receivedAndDeletedBlocks) throws IOException {
|
||||||
|
verifyRequest(nodeReg);
|
||||||
|
if(stateChangeLog.isDebugEnabled()) {
|
||||||
|
stateChangeLog.debug("*BLOCK* NameNode.blockReceivedAndDeleted: "
|
||||||
|
+"from "+nodeReg.getName()+" "+receivedAndDeletedBlocks.length
|
||||||
|
+" blocks.");
|
||||||
|
}
|
||||||
|
namesystem.getBlockManager().blockReceivedAndDeleted(
|
||||||
|
nodeReg, poolId, receivedAndDeletedBlocks);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // DatanodeProtocol
|
||||||
|
public void errorReport(DatanodeRegistration nodeReg,
|
||||||
|
int errorCode, String msg) throws IOException {
|
||||||
|
String dnName = (nodeReg == null ? "unknown DataNode" : nodeReg.getName());
|
||||||
|
|
||||||
|
if (errorCode == DatanodeProtocol.NOTIFY) {
|
||||||
|
LOG.info("Error report from " + dnName + ": " + msg);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
verifyRequest(nodeReg);
|
||||||
|
|
||||||
|
if (errorCode == DatanodeProtocol.DISK_ERROR) {
|
||||||
|
LOG.warn("Disk error on " + dnName + ": " + msg);
|
||||||
|
} else if (errorCode == DatanodeProtocol.FATAL_DISK_ERROR) {
|
||||||
|
LOG.warn("Fatal disk error on " + dnName + ": " + msg);
|
||||||
|
namesystem.getBlockManager().getDatanodeManager().removeDatanode(nodeReg);
|
||||||
|
} else {
|
||||||
|
LOG.info("Error report from " + dnName + ": " + msg);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // DatanodeProtocol, NamenodeProtocol
|
||||||
|
public NamespaceInfo versionRequest() throws IOException {
|
||||||
|
return namesystem.getNamespaceInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // DatanodeProtocol
|
||||||
|
public UpgradeCommand processUpgradeCommand(UpgradeCommand comm) throws IOException {
|
||||||
|
return namesystem.processDistributedUpgradeCommand(comm);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Verify request.
|
||||||
|
*
|
||||||
|
* Verifies correctness of the datanode version, registration ID, and
|
||||||
|
* if the datanode does not need to be shutdown.
|
||||||
|
*
|
||||||
|
* @param nodeReg data node registration
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
void verifyRequest(NodeRegistration nodeReg) throws IOException {
|
||||||
|
verifyVersion(nodeReg.getVersion());
|
||||||
|
if (!namesystem.getRegistrationID().equals(nodeReg.getRegistrationID())) {
|
||||||
|
LOG.warn("Invalid registrationID - expected: "
|
||||||
|
+ namesystem.getRegistrationID() + " received: "
|
||||||
|
+ nodeReg.getRegistrationID());
|
||||||
|
throw new UnregisteredNodeException(nodeReg);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // RefreshAuthorizationPolicyProtocol
|
||||||
|
public void refreshServiceAcl() throws IOException {
|
||||||
|
if (!serviceAuthEnabled) {
|
||||||
|
throw new AuthorizationException("Service Level Authorization not enabled!");
|
||||||
|
}
|
||||||
|
|
||||||
|
this.server.refreshServiceAcl(new Configuration(), new HDFSPolicyProvider());
|
||||||
|
if (this.serviceRpcServer != null) {
|
||||||
|
this.serviceRpcServer.refreshServiceAcl(new Configuration(), new HDFSPolicyProvider());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // RefreshAuthorizationPolicyProtocol
|
||||||
|
public void refreshUserToGroupsMappings() throws IOException {
|
||||||
|
LOG.info("Refreshing all user-to-groups mappings. Requested by user: " +
|
||||||
|
UserGroupInformation.getCurrentUser().getShortUserName());
|
||||||
|
Groups.getUserToGroupsMappingService().refresh();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // RefreshAuthorizationPolicyProtocol
|
||||||
|
public void refreshSuperUserGroupsConfiguration() {
|
||||||
|
LOG.info("Refreshing SuperUser proxy group mapping list ");
|
||||||
|
|
||||||
|
ProxyUsers.refreshSuperUserGroupsConfiguration();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // GetUserMappingsProtocol
|
||||||
|
public String[] getGroupsForUser(String user) throws IOException {
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Getting groups for user " + user);
|
||||||
|
}
|
||||||
|
return UserGroupInformation.createRemoteUser(user).getGroupNames();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Verify version.
|
||||||
|
*
|
||||||
|
* @param version
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
void verifyVersion(int version) throws IOException {
|
||||||
|
if (version != HdfsConstants.LAYOUT_VERSION)
|
||||||
|
throw new IncorrectVersionException(version, "data node");
|
||||||
|
}
|
||||||
|
|
||||||
|
private static String getClientMachine() {
|
||||||
|
String clientMachine = Server.getRemoteAddress();
|
||||||
|
if (clientMachine == null) {
|
||||||
|
clientMachine = "";
|
||||||
|
}
|
||||||
|
return clientMachine;
|
||||||
|
}
|
||||||
|
}
|
@ -173,7 +173,7 @@ public void fsck() {
|
|||||||
out.println(msg);
|
out.println(msg);
|
||||||
namenode.getNamesystem().logFsckEvent(path, remoteAddress);
|
namenode.getNamesystem().logFsckEvent(path, remoteAddress);
|
||||||
|
|
||||||
final HdfsFileStatus file = namenode.getFileInfo(path);
|
final HdfsFileStatus file = namenode.getRpcServer().getFileInfo(path);
|
||||||
if (file != null) {
|
if (file != null) {
|
||||||
|
|
||||||
if (showCorruptFileBlocks) {
|
if (showCorruptFileBlocks) {
|
||||||
@ -250,7 +250,8 @@ private void check(String parent, HdfsFileStatus file, Result res) throws IOExce
|
|||||||
res.totalDirs++;
|
res.totalDirs++;
|
||||||
do {
|
do {
|
||||||
assert lastReturnedName != null;
|
assert lastReturnedName != null;
|
||||||
thisListing = namenode.getListing(path, lastReturnedName, false);
|
thisListing = namenode.getRpcServer().getListing(
|
||||||
|
path, lastReturnedName, false);
|
||||||
if (thisListing == null) {
|
if (thisListing == null) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
@ -385,7 +386,7 @@ private void check(String parent, HdfsFileStatus file, Result res) throws IOExce
|
|||||||
break;
|
break;
|
||||||
case FIXING_DELETE:
|
case FIXING_DELETE:
|
||||||
if (!isOpen)
|
if (!isOpen)
|
||||||
namenode.delete(path, true);
|
namenode.getRpcServer().delete(path, true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (showFiles) {
|
if (showFiles) {
|
||||||
@ -414,7 +415,8 @@ private void lostFoundMove(String parent, HdfsFileStatus file, LocatedBlocks blo
|
|||||||
String target = lostFound + fullName;
|
String target = lostFound + fullName;
|
||||||
String errmsg = "Failed to move " + fullName + " to /lost+found";
|
String errmsg = "Failed to move " + fullName + " to /lost+found";
|
||||||
try {
|
try {
|
||||||
if (!namenode.mkdirs(target, file.getPermission(), true)) {
|
if (!namenode.getRpcServer().mkdirs(
|
||||||
|
target, file.getPermission(), true)) {
|
||||||
LOG.warn(errmsg);
|
LOG.warn(errmsg);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
@ -49,6 +49,7 @@
|
|||||||
import org.apache.hadoop.hdfs.server.common.Storage;
|
import org.apache.hadoop.hdfs.server.common.Storage;
|
||||||
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
||||||
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
|
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.net.NodeBase;
|
import org.apache.hadoop.net.NodeBase;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
@ -354,7 +355,7 @@ void generateHealthReport(JspWriter out, NameNode nn,
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static String getDelegationToken(final NameNode nn,
|
static String getDelegationToken(final NamenodeProtocols nn,
|
||||||
HttpServletRequest request, Configuration conf,
|
HttpServletRequest request, Configuration conf,
|
||||||
final UserGroupInformation ugi) throws IOException, InterruptedException {
|
final UserGroupInformation ugi) throws IOException, InterruptedException {
|
||||||
Token<DelegationTokenIdentifier> token = ugi
|
Token<DelegationTokenIdentifier> token = ugi
|
||||||
@ -381,7 +382,8 @@ static void redirectToRandomDataNode(ServletContext context,
|
|||||||
.getAttribute(JspHelper.CURRENT_CONF);
|
.getAttribute(JspHelper.CURRENT_CONF);
|
||||||
final DatanodeID datanode = getRandomDatanode(nn);
|
final DatanodeID datanode = getRandomDatanode(nn);
|
||||||
UserGroupInformation ugi = JspHelper.getUGI(context, request, conf);
|
UserGroupInformation ugi = JspHelper.getUGI(context, request, conf);
|
||||||
String tokenString = getDelegationToken(nn, request, conf, ugi);
|
String tokenString = getDelegationToken(
|
||||||
|
nn.getRpcServer(), request, conf, ugi);
|
||||||
// if the user is defined, get a delegation token and stringify it
|
// if the user is defined, get a delegation token and stringify it
|
||||||
final String redirectLocation;
|
final String redirectLocation;
|
||||||
final String nodeToRedirect;
|
final String nodeToRedirect;
|
||||||
|
@ -70,7 +70,7 @@ protected void doGet(final HttpServletRequest req, final HttpServletResponse res
|
|||||||
try {
|
try {
|
||||||
long result = ugi.doAs(new PrivilegedExceptionAction<Long>() {
|
long result = ugi.doAs(new PrivilegedExceptionAction<Long>() {
|
||||||
public Long run() throws Exception {
|
public Long run() throws Exception {
|
||||||
return nn.renewDelegationToken(token);
|
return nn.getRpcServer().renewDelegationToken(token);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
PrintStream os = new PrintStream(resp.getOutputStream());
|
PrintStream os = new PrintStream(resp.getOutputStream());
|
||||||
|
@ -1025,6 +1025,14 @@ public NameNode getNameNode() {
|
|||||||
return getNameNode(0);
|
return getNameNode(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get an instance of the NameNode's RPC handler.
|
||||||
|
*/
|
||||||
|
public NamenodeProtocols getNameNodeRpc() {
|
||||||
|
checkSingleNameNode();
|
||||||
|
return getNameNode(0).getRpcServer();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the NameNode for the index. May be null.
|
* Gets the NameNode for the index. May be null.
|
||||||
*/
|
*/
|
||||||
@ -1361,7 +1369,15 @@ public boolean isNameNodeUp(int nnIndex) {
|
|||||||
if (nameNode == null) {
|
if (nameNode == null) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
long[] sizes = nameNode.getStats();
|
long[] sizes;
|
||||||
|
try {
|
||||||
|
sizes = nameNode.getRpcServer().getStats();
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
// This method above should never throw.
|
||||||
|
// It only throws IOE since it is exposed via RPC
|
||||||
|
throw new AssertionError("Unexpected IOE thrown: "
|
||||||
|
+ StringUtils.stringifyException(ioe));
|
||||||
|
}
|
||||||
boolean isUp = false;
|
boolean isUp = false;
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
isUp = ((!nameNode.isInSafeMode() || !waitSafeMode) && sizes[0] != 0);
|
isUp = ((!nameNode.isInSafeMode() || !waitSafeMode) && sizes[0] != 0);
|
||||||
|
@ -26,6 +26,7 @@
|
|||||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
|
import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
|
||||||
@ -45,7 +46,7 @@ public class TestClientProtocolForPipelineRecovery {
|
|||||||
try {
|
try {
|
||||||
cluster.waitActive();
|
cluster.waitActive();
|
||||||
FileSystem fileSys = cluster.getFileSystem();
|
FileSystem fileSys = cluster.getFileSystem();
|
||||||
NameNode namenode = cluster.getNameNode();
|
NamenodeProtocols namenode = cluster.getNameNodeRpc();
|
||||||
|
|
||||||
/* Test writing to finalized replicas */
|
/* Test writing to finalized replicas */
|
||||||
Path file = new Path("dataprotocol.dat");
|
Path file = new Path("dataprotocol.dat");
|
||||||
|
@ -57,6 +57,7 @@
|
|||||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
|
import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.ipc.RemoteException;
|
import org.apache.hadoop.ipc.RemoteException;
|
||||||
import org.apache.hadoop.ipc.Client;
|
import org.apache.hadoop.ipc.Client;
|
||||||
@ -190,7 +191,7 @@ public void testNotYetReplicatedErrors() throws IOException
|
|||||||
final int maxRetries = 1; // Allow one retry (total of two calls)
|
final int maxRetries = 1; // Allow one retry (total of two calls)
|
||||||
conf.setInt(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY, maxRetries);
|
conf.setInt(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY, maxRetries);
|
||||||
|
|
||||||
NameNode mockNN = mock(NameNode.class);
|
NamenodeProtocols mockNN = mock(NamenodeProtocols.class);
|
||||||
Answer<Object> answer = new ThrowsException(new IOException()) {
|
Answer<Object> answer = new ThrowsException(new IOException()) {
|
||||||
int retryCount = 0;
|
int retryCount = 0;
|
||||||
|
|
||||||
@ -240,8 +241,8 @@ public void testFailuresArePerOperation() throws Exception
|
|||||||
try {
|
try {
|
||||||
cluster.waitActive();
|
cluster.waitActive();
|
||||||
FileSystem fs = cluster.getFileSystem();
|
FileSystem fs = cluster.getFileSystem();
|
||||||
NameNode preSpyNN = cluster.getNameNode();
|
NamenodeProtocols preSpyNN = cluster.getNameNodeRpc();
|
||||||
NameNode spyNN = spy(preSpyNN);
|
NamenodeProtocols spyNN = spy(preSpyNN);
|
||||||
DFSClient client = new DFSClient(null, spyNN, conf, null);
|
DFSClient client = new DFSClient(null, spyNN, conf, null);
|
||||||
int maxBlockAcquires = client.getMaxBlockAcquireFailures();
|
int maxBlockAcquires = client.getMaxBlockAcquireFailures();
|
||||||
assertTrue(maxBlockAcquires > 0);
|
assertTrue(maxBlockAcquires > 0);
|
||||||
@ -305,11 +306,11 @@ public void testFailuresArePerOperation() throws Exception
|
|||||||
*/
|
*/
|
||||||
private static class FailNTimesAnswer implements Answer<LocatedBlocks> {
|
private static class FailNTimesAnswer implements Answer<LocatedBlocks> {
|
||||||
private int failuresLeft;
|
private int failuresLeft;
|
||||||
private NameNode realNN;
|
private NamenodeProtocols realNN;
|
||||||
|
|
||||||
public FailNTimesAnswer(NameNode realNN, int timesToFail) {
|
public FailNTimesAnswer(NamenodeProtocols preSpyNN, int timesToFail) {
|
||||||
failuresLeft = timesToFail;
|
failuresLeft = timesToFail;
|
||||||
this.realNN = realNN;
|
this.realNN = preSpyNN;
|
||||||
}
|
}
|
||||||
|
|
||||||
public LocatedBlocks answer(InvocationOnMock invocation) throws IOException {
|
public LocatedBlocks answer(InvocationOnMock invocation) throws IOException {
|
||||||
@ -603,7 +604,8 @@ public void testGetFileChecksum() throws Exception {
|
|||||||
|
|
||||||
//stop the first datanode
|
//stop the first datanode
|
||||||
final List<LocatedBlock> locatedblocks = DFSClient.callGetBlockLocations(
|
final List<LocatedBlock> locatedblocks = DFSClient.callGetBlockLocations(
|
||||||
cluster.getNameNode(), f, 0, Long.MAX_VALUE).getLocatedBlocks();
|
cluster.getNameNodeRpc(), f, 0, Long.MAX_VALUE)
|
||||||
|
.getLocatedBlocks();
|
||||||
final DatanodeInfo first = locatedblocks.get(0).getLocations()[0];
|
final DatanodeInfo first = locatedblocks.get(0).getLocations()[0];
|
||||||
cluster.stopDataNode(first.getName());
|
cluster.stopDataNode(first.getName());
|
||||||
|
|
||||||
|
@ -293,10 +293,11 @@ static void refreshNodes(final FSNamesystem ns, final Configuration conf
|
|||||||
}
|
}
|
||||||
|
|
||||||
private void verifyStats(NameNode namenode, FSNamesystem fsn,
|
private void verifyStats(NameNode namenode, FSNamesystem fsn,
|
||||||
DatanodeInfo node, boolean decommissioning) throws InterruptedException {
|
DatanodeInfo node, boolean decommissioning)
|
||||||
|
throws InterruptedException, IOException {
|
||||||
// Do the stats check over 10 iterations
|
// Do the stats check over 10 iterations
|
||||||
for (int i = 0; i < 10; i++) {
|
for (int i = 0; i < 10; i++) {
|
||||||
long[] newStats = namenode.getStats();
|
long[] newStats = namenode.getRpcServer().getStats();
|
||||||
|
|
||||||
// For decommissioning nodes, ensure capacity of the DN is no longer
|
// For decommissioning nodes, ensure capacity of the DN is no longer
|
||||||
// counted. Only used space of the DN is counted in cluster capacity
|
// counted. Only used space of the DN is counted in cluster capacity
|
||||||
|
@ -36,7 +36,6 @@
|
|||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hdfs.protocol.Block;
|
|
||||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||||
@ -44,6 +43,7 @@
|
|||||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
|
import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.apache.log4j.Level;
|
import org.apache.log4j.Level;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
@ -151,8 +151,8 @@ public void testRecoverFinalizedBlock() throws Throwable {
|
|||||||
|
|
||||||
try {
|
try {
|
||||||
cluster.waitActive();
|
cluster.waitActive();
|
||||||
NameNode preSpyNN = cluster.getNameNode();
|
NamenodeProtocols preSpyNN = cluster.getNameNodeRpc();
|
||||||
NameNode spyNN = spy(preSpyNN);
|
NamenodeProtocols spyNN = spy(preSpyNN);
|
||||||
|
|
||||||
// Delay completeFile
|
// Delay completeFile
|
||||||
GenericTestUtils.DelayAnswer delayer = new GenericTestUtils.DelayAnswer(LOG);
|
GenericTestUtils.DelayAnswer delayer = new GenericTestUtils.DelayAnswer(LOG);
|
||||||
@ -222,8 +222,8 @@ public void testCompleteOtherLeaseHoldersFile() throws Throwable {
|
|||||||
|
|
||||||
try {
|
try {
|
||||||
cluster.waitActive();
|
cluster.waitActive();
|
||||||
NameNode preSpyNN = cluster.getNameNode();
|
NamenodeProtocols preSpyNN = cluster.getNameNodeRpc();
|
||||||
NameNode spyNN = spy(preSpyNN);
|
NamenodeProtocols spyNN = spy(preSpyNN);
|
||||||
|
|
||||||
// Delay completeFile
|
// Delay completeFile
|
||||||
GenericTestUtils.DelayAnswer delayer =
|
GenericTestUtils.DelayAnswer delayer =
|
||||||
|
@ -420,7 +420,7 @@ public void testFileCreationError3() throws IOException {
|
|||||||
final Path f = new Path("/foo.txt");
|
final Path f = new Path("/foo.txt");
|
||||||
createFile(dfs, f, 3);
|
createFile(dfs, f, 3);
|
||||||
try {
|
try {
|
||||||
cluster.getNameNode().addBlock(f.toString(),
|
cluster.getNameNodeRpc().addBlock(f.toString(),
|
||||||
client.clientName, null, null);
|
client.clientName, null, null);
|
||||||
fail();
|
fail();
|
||||||
} catch(IOException ioe) {
|
} catch(IOException ioe) {
|
||||||
|
@ -106,7 +106,7 @@ public void testBlockSynchronization() throws Exception {
|
|||||||
|
|
||||||
|
|
||||||
DataNode.LOG.info("dfs.dfs.clientName=" + dfs.dfs.clientName);
|
DataNode.LOG.info("dfs.dfs.clientName=" + dfs.dfs.clientName);
|
||||||
cluster.getNameNode().append(filestr, dfs.dfs.clientName);
|
cluster.getNameNodeRpc().append(filestr, dfs.dfs.clientName);
|
||||||
|
|
||||||
// expire lease to trigger block recovery.
|
// expire lease to trigger block recovery.
|
||||||
waitLeaseRecovery(cluster);
|
waitLeaseRecovery(cluster);
|
||||||
@ -129,14 +129,14 @@ public void testBlockSynchronization() throws Exception {
|
|||||||
filestr = "/foo.safemode";
|
filestr = "/foo.safemode";
|
||||||
filepath = new Path(filestr);
|
filepath = new Path(filestr);
|
||||||
dfs.create(filepath, (short)1);
|
dfs.create(filepath, (short)1);
|
||||||
cluster.getNameNode().setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
|
cluster.getNameNodeRpc().setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
|
||||||
assertTrue(dfs.dfs.exists(filestr));
|
assertTrue(dfs.dfs.exists(filestr));
|
||||||
DFSTestUtil.waitReplication(dfs, filepath, (short)1);
|
DFSTestUtil.waitReplication(dfs, filepath, (short)1);
|
||||||
waitLeaseRecovery(cluster);
|
waitLeaseRecovery(cluster);
|
||||||
// verify that we still cannot recover the lease
|
// verify that we still cannot recover the lease
|
||||||
LeaseManager lm = NameNodeAdapter.getLeaseManager(cluster.getNamesystem());
|
LeaseManager lm = NameNodeAdapter.getLeaseManager(cluster.getNamesystem());
|
||||||
assertTrue("Found " + lm.countLease() + " lease, expected 1", lm.countLease() == 1);
|
assertTrue("Found " + lm.countLease() + " lease, expected 1", lm.countLease() == 1);
|
||||||
cluster.getNameNode().setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
cluster.getNameNodeRpc().setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
if (cluster != null) {cluster.shutdown();}
|
if (cluster != null) {cluster.shutdown();}
|
||||||
|
@ -100,7 +100,7 @@ public void pipeline_01() throws IOException {
|
|||||||
ofs.writeBytes("Some more stuff to write");
|
ofs.writeBytes("Some more stuff to write");
|
||||||
((DFSOutputStream) ofs.getWrappedStream()).hflush();
|
((DFSOutputStream) ofs.getWrappedStream()).hflush();
|
||||||
|
|
||||||
List<LocatedBlock> lb = cluster.getNameNode().getBlockLocations(
|
List<LocatedBlock> lb = cluster.getNameNodeRpc().getBlockLocations(
|
||||||
filePath.toString(), FILE_SIZE - 1, FILE_SIZE).getLocatedBlocks();
|
filePath.toString(), FILE_SIZE - 1, FILE_SIZE).getLocatedBlocks();
|
||||||
|
|
||||||
String bpid = cluster.getNamesystem().getBlockPoolId();
|
String bpid = cluster.getNamesystem().getBlockPoolId();
|
||||||
|
@ -51,7 +51,7 @@
|
|||||||
import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
|
import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
|
||||||
import org.apache.hadoop.hdfs.server.datanode.DataStorage;
|
import org.apache.hadoop.hdfs.server.datanode.DataStorage;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NNStorage;
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class defines a number of static helper methods used by the
|
* This class defines a number of static helper methods used by the
|
||||||
@ -121,7 +121,7 @@ public static void initialize() throws Exception {
|
|||||||
.manageNameDfsDirs(false)
|
.manageNameDfsDirs(false)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
NameNode namenode = cluster.getNameNode();
|
NamenodeProtocols namenode = cluster.getNameNodeRpc();
|
||||||
namenodeStorageNamespaceID = namenode.versionRequest().getNamespaceID();
|
namenodeStorageNamespaceID = namenode.versionRequest().getNamespaceID();
|
||||||
namenodeStorageFsscTime = namenode.versionRequest().getCTime();
|
namenodeStorageFsscTime = namenode.versionRequest().getCTime();
|
||||||
namenodeStorageClusterID = namenode.versionRequest().getClusterID();
|
namenodeStorageClusterID = namenode.versionRequest().getClusterID();
|
||||||
@ -517,7 +517,7 @@ public static int getCurrentLayoutVersion() {
|
|||||||
*/
|
*/
|
||||||
public static int getCurrentNamespaceID(MiniDFSCluster cluster) throws IOException {
|
public static int getCurrentNamespaceID(MiniDFSCluster cluster) throws IOException {
|
||||||
if (cluster != null) {
|
if (cluster != null) {
|
||||||
return cluster.getNameNode().versionRequest().getNamespaceID();
|
return cluster.getNameNodeRpc().versionRequest().getNamespaceID();
|
||||||
}
|
}
|
||||||
return namenodeStorageNamespaceID;
|
return namenodeStorageNamespaceID;
|
||||||
}
|
}
|
||||||
@ -528,7 +528,7 @@ public static int getCurrentNamespaceID(MiniDFSCluster cluster) throws IOExcepti
|
|||||||
*/
|
*/
|
||||||
public static String getCurrentClusterID(MiniDFSCluster cluster) throws IOException {
|
public static String getCurrentClusterID(MiniDFSCluster cluster) throws IOException {
|
||||||
if (cluster != null) {
|
if (cluster != null) {
|
||||||
return cluster.getNameNode().versionRequest().getClusterID();
|
return cluster.getNameNodeRpc().versionRequest().getClusterID();
|
||||||
}
|
}
|
||||||
return namenodeStorageClusterID;
|
return namenodeStorageClusterID;
|
||||||
}
|
}
|
||||||
@ -539,7 +539,7 @@ public static String getCurrentClusterID(MiniDFSCluster cluster) throws IOExcept
|
|||||||
*/
|
*/
|
||||||
public static String getCurrentBlockPoolID(MiniDFSCluster cluster) throws IOException {
|
public static String getCurrentBlockPoolID(MiniDFSCluster cluster) throws IOException {
|
||||||
if (cluster != null) {
|
if (cluster != null) {
|
||||||
return cluster.getNameNode().versionRequest().getBlockPoolID();
|
return cluster.getNameNodeRpc().versionRequest().getBlockPoolID();
|
||||||
}
|
}
|
||||||
return namenodeStorageBlockPoolID;
|
return namenodeStorageBlockPoolID;
|
||||||
}
|
}
|
||||||
@ -554,7 +554,7 @@ public static String getCurrentBlockPoolID(MiniDFSCluster cluster) throws IOExce
|
|||||||
*/
|
*/
|
||||||
public static long getCurrentFsscTime(MiniDFSCluster cluster) throws IOException {
|
public static long getCurrentFsscTime(MiniDFSCluster cluster) throws IOException {
|
||||||
if (cluster != null) {
|
if (cluster != null) {
|
||||||
return cluster.getNameNode().versionRequest().getCTime();
|
return cluster.getNameNodeRpc().versionRequest().getCTime();
|
||||||
}
|
}
|
||||||
return namenodeStorageFsscTime;
|
return namenodeStorageFsscTime;
|
||||||
}
|
}
|
||||||
|
@ -375,11 +375,11 @@ public void testBlockTokenInLastLocatedBlock() throws IOException,
|
|||||||
Path filePath = new Path(fileName);
|
Path filePath = new Path(fileName);
|
||||||
FSDataOutputStream out = fs.create(filePath, (short) 1);
|
FSDataOutputStream out = fs.create(filePath, (short) 1);
|
||||||
out.write(new byte[1000]);
|
out.write(new byte[1000]);
|
||||||
LocatedBlocks locatedBlocks = cluster.getNameNode().getBlockLocations(
|
LocatedBlocks locatedBlocks = cluster.getNameNodeRpc().getBlockLocations(
|
||||||
fileName, 0, 1000);
|
fileName, 0, 1000);
|
||||||
while (locatedBlocks.getLastLocatedBlock() == null) {
|
while (locatedBlocks.getLastLocatedBlock() == null) {
|
||||||
Thread.sleep(100);
|
Thread.sleep(100);
|
||||||
locatedBlocks = cluster.getNameNode().getBlockLocations(fileName, 0,
|
locatedBlocks = cluster.getNameNodeRpc().getBlockLocations(fileName, 0,
|
||||||
1000);
|
1000);
|
||||||
}
|
}
|
||||||
Token<BlockTokenIdentifier> token = locatedBlocks.getLastLocatedBlock()
|
Token<BlockTokenIdentifier> token = locatedBlocks.getLastLocatedBlock()
|
||||||
|
@ -89,7 +89,7 @@ private static class Suite {
|
|||||||
this.cluster = cluster;
|
this.cluster = cluster;
|
||||||
clients = new ClientProtocol[nNameNodes];
|
clients = new ClientProtocol[nNameNodes];
|
||||||
for(int i = 0; i < nNameNodes; i++) {
|
for(int i = 0; i < nNameNodes; i++) {
|
||||||
clients[i] = cluster.getNameNode(i);
|
clients[i] = cluster.getNameNode(i).getRpcServer();
|
||||||
}
|
}
|
||||||
replication = (short)Math.max(1, nDataNodes - 1);
|
replication = (short)Math.max(1, nDataNodes - 1);
|
||||||
}
|
}
|
||||||
|
@ -51,6 +51,7 @@
|
|||||||
import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
|
import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
|
||||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.security.token.Token;
|
import org.apache.hadoop.security.token.Token;
|
||||||
import org.apache.hadoop.util.StringUtils;
|
import org.apache.hadoop.util.StringUtils;
|
||||||
@ -314,6 +315,7 @@ public void testRead() throws Exception {
|
|||||||
assertEquals(numDataNodes, cluster.getDataNodes().size());
|
assertEquals(numDataNodes, cluster.getDataNodes().size());
|
||||||
|
|
||||||
final NameNode nn = cluster.getNameNode();
|
final NameNode nn = cluster.getNameNode();
|
||||||
|
final NamenodeProtocols nnProto = nn.getRpcServer();
|
||||||
final BlockManager bm = nn.getNamesystem().getBlockManager();
|
final BlockManager bm = nn.getNamesystem().getBlockManager();
|
||||||
final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager();
|
final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager();
|
||||||
|
|
||||||
@ -344,7 +346,7 @@ public void testRead() throws Exception {
|
|||||||
|
|
||||||
new DFSClient(new InetSocketAddress("localhost",
|
new DFSClient(new InetSocketAddress("localhost",
|
||||||
cluster.getNameNodePort()), conf);
|
cluster.getNameNodePort()), conf);
|
||||||
List<LocatedBlock> locatedBlocks = nn.getBlockLocations(
|
List<LocatedBlock> locatedBlocks = nnProto.getBlockLocations(
|
||||||
FILE_TO_READ, 0, FILE_SIZE).getLocatedBlocks();
|
FILE_TO_READ, 0, FILE_SIZE).getLocatedBlocks();
|
||||||
LocatedBlock lblock = locatedBlocks.get(0); // first block
|
LocatedBlock lblock = locatedBlocks.get(0); // first block
|
||||||
Token<BlockTokenIdentifier> myToken = lblock.getBlockToken();
|
Token<BlockTokenIdentifier> myToken = lblock.getBlockToken();
|
||||||
|
@ -34,7 +34,6 @@
|
|||||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
||||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
@ -139,7 +138,7 @@ public void blockReport_01() throws IOException {
|
|||||||
DataNode dn = cluster.getDataNodes().get(DN_N0);
|
DataNode dn = cluster.getDataNodes().get(DN_N0);
|
||||||
String poolId = cluster.getNamesystem().getBlockPoolId();
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
||||||
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
||||||
cluster.getNameNode().blockReport(dnR, poolId,
|
cluster.getNameNodeRpc().blockReport(dnR, poolId,
|
||||||
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
||||||
|
|
||||||
List<LocatedBlock> blocksAfterReport =
|
List<LocatedBlock> blocksAfterReport =
|
||||||
@ -181,7 +180,8 @@ public void blockReport_02() throws IOException {
|
|||||||
|
|
||||||
List<ExtendedBlock> blocks2Remove = new ArrayList<ExtendedBlock>();
|
List<ExtendedBlock> blocks2Remove = new ArrayList<ExtendedBlock>();
|
||||||
List<Integer> removedIndex = new ArrayList<Integer>();
|
List<Integer> removedIndex = new ArrayList<Integer>();
|
||||||
List<LocatedBlock> lBlocks = cluster.getNameNode().getBlockLocations(
|
List<LocatedBlock> lBlocks =
|
||||||
|
cluster.getNameNodeRpc().getBlockLocations(
|
||||||
filePath.toString(), FILE_START,
|
filePath.toString(), FILE_START,
|
||||||
FILE_SIZE).getLocatedBlocks();
|
FILE_SIZE).getLocatedBlocks();
|
||||||
|
|
||||||
@ -218,7 +218,7 @@ public void blockReport_02() throws IOException {
|
|||||||
DataNode dn = cluster.getDataNodes().get(DN_N0);
|
DataNode dn = cluster.getDataNodes().get(DN_N0);
|
||||||
String poolId = cluster.getNamesystem().getBlockPoolId();
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
||||||
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
||||||
cluster.getNameNode().blockReport(dnR, poolId,
|
cluster.getNameNodeRpc().blockReport(dnR, poolId,
|
||||||
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
||||||
|
|
||||||
BlockManagerTestUtil.getComputedDatanodeWork(cluster.getNamesystem()
|
BlockManagerTestUtil.getComputedDatanodeWork(cluster.getNamesystem()
|
||||||
@ -258,7 +258,8 @@ public void blockReport_03() throws IOException {
|
|||||||
DataNode dn = cluster.getDataNodes().get(DN_N0);
|
DataNode dn = cluster.getDataNodes().get(DN_N0);
|
||||||
String poolId = cluster.getNamesystem().getBlockPoolId();
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
||||||
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
||||||
DatanodeCommand dnCmd = cluster.getNameNode().blockReport(dnR, poolId,
|
DatanodeCommand dnCmd =
|
||||||
|
cluster.getNameNodeRpc().blockReport(dnR, poolId,
|
||||||
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
||||||
if(LOG.isDebugEnabled()) {
|
if(LOG.isDebugEnabled()) {
|
||||||
LOG.debug("Got the command: " + dnCmd);
|
LOG.debug("Got the command: " + dnCmd);
|
||||||
@ -310,7 +311,7 @@ public void blockReport_06() throws IOException {
|
|||||||
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
||||||
String poolId = cluster.getNamesystem().getBlockPoolId();
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
||||||
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
||||||
cluster.getNameNode().blockReport(dnR, poolId,
|
cluster.getNameNodeRpc().blockReport(dnR, poolId,
|
||||||
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
||||||
printStats();
|
printStats();
|
||||||
assertEquals("Wrong number of PendingReplication Blocks",
|
assertEquals("Wrong number of PendingReplication Blocks",
|
||||||
@ -359,7 +360,7 @@ public void blockReport_07() throws IOException {
|
|||||||
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
||||||
String poolId = cluster.getNamesystem().getBlockPoolId();
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
||||||
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
||||||
cluster.getNameNode().blockReport(dnR, poolId,
|
cluster.getNameNodeRpc().blockReport(dnR, poolId,
|
||||||
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
||||||
printStats();
|
printStats();
|
||||||
assertEquals("Wrong number of Corrupted blocks",
|
assertEquals("Wrong number of Corrupted blocks",
|
||||||
@ -381,7 +382,7 @@ public void blockReport_07() throws IOException {
|
|||||||
LOG.debug("Done corrupting length of " + corruptedBlock.getBlockName());
|
LOG.debug("Done corrupting length of " + corruptedBlock.getBlockName());
|
||||||
}
|
}
|
||||||
|
|
||||||
cluster.getNameNode().blockReport(dnR, poolId,
|
cluster.getNameNodeRpc().blockReport(dnR, poolId,
|
||||||
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
||||||
printStats();
|
printStats();
|
||||||
|
|
||||||
@ -431,7 +432,7 @@ public void blockReport_08() throws IOException {
|
|||||||
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
||||||
String poolId = cluster.getNamesystem().getBlockPoolId();
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
||||||
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
||||||
cluster.getNameNode().blockReport(dnR, poolId,
|
cluster.getNameNodeRpc().blockReport(dnR, poolId,
|
||||||
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
||||||
printStats();
|
printStats();
|
||||||
assertEquals("Wrong number of PendingReplication blocks",
|
assertEquals("Wrong number of PendingReplication blocks",
|
||||||
@ -477,7 +478,7 @@ public void blockReport_09() throws IOException {
|
|||||||
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
||||||
String poolId = cluster.getNamesystem().getBlockPoolId();
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
||||||
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
||||||
cluster.getNameNode().blockReport(dnR, poolId,
|
cluster.getNameNodeRpc().blockReport(dnR, poolId,
|
||||||
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
||||||
printStats();
|
printStats();
|
||||||
assertEquals("Wrong number of PendingReplication blocks",
|
assertEquals("Wrong number of PendingReplication blocks",
|
||||||
@ -590,7 +591,7 @@ private ArrayList<Block> prepareForRide(final Path filePath,
|
|||||||
DFSTestUtil.createFile(fs, filePath, fileSize,
|
DFSTestUtil.createFile(fs, filePath, fileSize,
|
||||||
REPL_FACTOR, rand.nextLong());
|
REPL_FACTOR, rand.nextLong());
|
||||||
|
|
||||||
return locatedToBlocks(cluster.getNameNode()
|
return locatedToBlocks(cluster.getNameNodeRpc()
|
||||||
.getBlockLocations(filePath.toString(), FILE_START,
|
.getBlockLocations(filePath.toString(), FILE_START,
|
||||||
fileSize).getLocatedBlocks(), null);
|
fileSize).getLocatedBlocks(), null);
|
||||||
}
|
}
|
||||||
@ -707,7 +708,8 @@ private void corruptBlockGS(final Block block)
|
|||||||
private Block findBlock(Path path, long size) throws IOException {
|
private Block findBlock(Path path, long size) throws IOException {
|
||||||
Block ret;
|
Block ret;
|
||||||
List<LocatedBlock> lbs =
|
List<LocatedBlock> lbs =
|
||||||
cluster.getNameNode().getBlockLocations(path.toString(),
|
cluster.getNameNodeRpc()
|
||||||
|
.getBlockLocations(path.toString(),
|
||||||
FILE_START, size).getLocatedBlocks();
|
FILE_START, size).getLocatedBlocks();
|
||||||
LocatedBlock lb = lbs.get(lbs.size() - 1);
|
LocatedBlock lb = lbs.get(lbs.size() - 1);
|
||||||
|
|
||||||
|
@ -40,8 +40,8 @@
|
|||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
||||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
||||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
|
|
||||||
@ -144,7 +144,7 @@ public void testVolumeFailure() throws IOException {
|
|||||||
String bpid = cluster.getNamesystem().getBlockPoolId();
|
String bpid = cluster.getNamesystem().getBlockPoolId();
|
||||||
DatanodeRegistration dnR = dn.getDNRegistrationForBP(bpid);
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(bpid);
|
||||||
long[] bReport = dn.getFSDataset().getBlockReport(bpid).getBlockListAsLongs();
|
long[] bReport = dn.getFSDataset().getBlockReport(bpid).getBlockListAsLongs();
|
||||||
cluster.getNameNode().blockReport(dnR, bpid, bReport);
|
cluster.getNameNodeRpc().blockReport(dnR, bpid, bReport);
|
||||||
|
|
||||||
// verify number of blocks and files...
|
// verify number of blocks and files...
|
||||||
verify(filename, filesize);
|
verify(filename, filesize);
|
||||||
@ -216,7 +216,7 @@ private void verify(String fn, int fs) throws IOException{
|
|||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private void triggerFailure(String path, long size) throws IOException {
|
private void triggerFailure(String path, long size) throws IOException {
|
||||||
NameNode nn = cluster.getNameNode();
|
NamenodeProtocols nn = cluster.getNameNodeRpc();
|
||||||
List<LocatedBlock> locatedBlocks =
|
List<LocatedBlock> locatedBlocks =
|
||||||
nn.getBlockLocations(path, 0, size).getLocatedBlocks();
|
nn.getBlockLocations(path, 0, size).getLocatedBlocks();
|
||||||
|
|
||||||
@ -291,7 +291,7 @@ private int countNNBlocks(Map<String, BlockLocs> map, String path, long size)
|
|||||||
throws IOException {
|
throws IOException {
|
||||||
int total = 0;
|
int total = 0;
|
||||||
|
|
||||||
NameNode nn = cluster.getNameNode();
|
NamenodeProtocols nn = cluster.getNameNodeRpc();
|
||||||
List<LocatedBlock> locatedBlocks =
|
List<LocatedBlock> locatedBlocks =
|
||||||
nn.getBlockLocations(path, 0, size).getLocatedBlocks();
|
nn.getBlockLocations(path, 0, size).getLocatedBlocks();
|
||||||
//System.out.println("Number of blocks: " + locatedBlocks.size());
|
//System.out.println("Number of blocks: " + locatedBlocks.size());
|
||||||
|
@ -109,7 +109,7 @@ public void testTransferRbw() throws Exception {
|
|||||||
|
|
||||||
final DatanodeInfo oldnodeinfo;
|
final DatanodeInfo oldnodeinfo;
|
||||||
{
|
{
|
||||||
final DatanodeInfo[] datatnodeinfos = cluster.getNameNode(
|
final DatanodeInfo[] datatnodeinfos = cluster.getNameNodeRpc(
|
||||||
).getDatanodeReport(DatanodeReportType.LIVE);
|
).getDatanodeReport(DatanodeReportType.LIVE);
|
||||||
Assert.assertEquals(2, datatnodeinfos.length);
|
Assert.assertEquals(2, datatnodeinfos.length);
|
||||||
int i = 0;
|
int i = 0;
|
||||||
|
@ -46,6 +46,7 @@
|
|||||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
|
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
|
||||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||||
@ -101,6 +102,7 @@ public class NNThroughputBenchmark {
|
|||||||
|
|
||||||
static Configuration config;
|
static Configuration config;
|
||||||
static NameNode nameNode;
|
static NameNode nameNode;
|
||||||
|
static NamenodeProtocols nameNodeProto;
|
||||||
|
|
||||||
NNThroughputBenchmark(Configuration conf) throws IOException, LoginException {
|
NNThroughputBenchmark(Configuration conf) throws IOException, LoginException {
|
||||||
config = conf;
|
config = conf;
|
||||||
@ -120,6 +122,7 @@ public class NNThroughputBenchmark {
|
|||||||
// Start the NameNode
|
// Start the NameNode
|
||||||
String[] argv = new String[] {};
|
String[] argv = new String[] {};
|
||||||
nameNode = NameNode.createNameNode(argv, config);
|
nameNode = NameNode.createNameNode(argv, config);
|
||||||
|
nameNodeProto = nameNode.getRpcServer();
|
||||||
}
|
}
|
||||||
|
|
||||||
void close() throws IOException {
|
void close() throws IOException {
|
||||||
@ -265,9 +268,9 @@ private boolean isInPorgress() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
void cleanUp() throws IOException {
|
void cleanUp() throws IOException {
|
||||||
nameNode.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
nameNodeProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
||||||
if(!keepResults)
|
if(!keepResults)
|
||||||
nameNode.delete(getBaseDir(), true);
|
nameNodeProto.delete(getBaseDir(), true);
|
||||||
}
|
}
|
||||||
|
|
||||||
int getNumOpsExecuted() {
|
int getNumOpsExecuted() {
|
||||||
@ -398,7 +401,7 @@ public String toString() {
|
|||||||
void benchmarkOne() throws IOException {
|
void benchmarkOne() throws IOException {
|
||||||
for(int idx = 0; idx < opsPerThread; idx++) {
|
for(int idx = 0; idx < opsPerThread; idx++) {
|
||||||
if((localNumOpsExecuted+1) % statsOp.ugcRefreshCount == 0)
|
if((localNumOpsExecuted+1) % statsOp.ugcRefreshCount == 0)
|
||||||
nameNode.refreshUserToGroupsMappings();
|
nameNodeProto.refreshUserToGroupsMappings();
|
||||||
long stat = statsOp.executeOp(daemonId, idx, arg1);
|
long stat = statsOp.executeOp(daemonId, idx, arg1);
|
||||||
localNumOpsExecuted++;
|
localNumOpsExecuted++;
|
||||||
localCumulativeTime += stat;
|
localCumulativeTime += stat;
|
||||||
@ -459,9 +462,9 @@ String getExecutionArgument(int daemonId) {
|
|||||||
*/
|
*/
|
||||||
long executeOp(int daemonId, int inputIdx, String ignore)
|
long executeOp(int daemonId, int inputIdx, String ignore)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
nameNode.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
nameNodeProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
||||||
long start = System.currentTimeMillis();
|
long start = System.currentTimeMillis();
|
||||||
nameNode.delete(BASE_DIR_NAME, true);
|
nameNodeProto.delete(BASE_DIR_NAME, true);
|
||||||
long end = System.currentTimeMillis();
|
long end = System.currentTimeMillis();
|
||||||
return end-start;
|
return end-start;
|
||||||
}
|
}
|
||||||
@ -523,7 +526,7 @@ void parseArguments(List<String> args) {
|
|||||||
|
|
||||||
void generateInputs(int[] opsPerThread) throws IOException {
|
void generateInputs(int[] opsPerThread) throws IOException {
|
||||||
assert opsPerThread.length == numThreads : "Error opsPerThread.length";
|
assert opsPerThread.length == numThreads : "Error opsPerThread.length";
|
||||||
nameNode.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
nameNodeProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
||||||
// int generatedFileIdx = 0;
|
// int generatedFileIdx = 0;
|
||||||
LOG.info("Generate " + numOpsRequired + " intputs for " + getOpName());
|
LOG.info("Generate " + numOpsRequired + " intputs for " + getOpName());
|
||||||
fileNames = new String[numThreads][];
|
fileNames = new String[numThreads][];
|
||||||
@ -555,12 +558,12 @@ long executeOp(int daemonId, int inputIdx, String clientName)
|
|||||||
throws IOException {
|
throws IOException {
|
||||||
long start = System.currentTimeMillis();
|
long start = System.currentTimeMillis();
|
||||||
// dummyActionNoSynch(fileIdx);
|
// dummyActionNoSynch(fileIdx);
|
||||||
nameNode.create(fileNames[daemonId][inputIdx], FsPermission.getDefault(),
|
nameNodeProto.create(fileNames[daemonId][inputIdx], FsPermission.getDefault(),
|
||||||
clientName, new EnumSetWritable<CreateFlag>(EnumSet
|
clientName, new EnumSetWritable<CreateFlag>(EnumSet
|
||||||
.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)), true, replication, BLOCK_SIZE);
|
.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)), true, replication, BLOCK_SIZE);
|
||||||
long end = System.currentTimeMillis();
|
long end = System.currentTimeMillis();
|
||||||
for(boolean written = !closeUponCreate; !written;
|
for(boolean written = !closeUponCreate; !written;
|
||||||
written = nameNode.complete(fileNames[daemonId][inputIdx],
|
written = nameNodeProto.complete(fileNames[daemonId][inputIdx],
|
||||||
clientName, null));
|
clientName, null));
|
||||||
return end-start;
|
return end-start;
|
||||||
}
|
}
|
||||||
@ -628,11 +631,11 @@ void generateInputs(int[] opsPerThread) throws IOException {
|
|||||||
}
|
}
|
||||||
// use the same files for open
|
// use the same files for open
|
||||||
super.generateInputs(opsPerThread);
|
super.generateInputs(opsPerThread);
|
||||||
if(nameNode.getFileInfo(opCreate.getBaseDir()) != null
|
if(nameNodeProto.getFileInfo(opCreate.getBaseDir()) != null
|
||||||
&& nameNode.getFileInfo(getBaseDir()) == null) {
|
&& nameNodeProto.getFileInfo(getBaseDir()) == null) {
|
||||||
nameNode.rename(opCreate.getBaseDir(), getBaseDir());
|
nameNodeProto.rename(opCreate.getBaseDir(), getBaseDir());
|
||||||
}
|
}
|
||||||
if(nameNode.getFileInfo(getBaseDir()) == null) {
|
if(nameNodeProto.getFileInfo(getBaseDir()) == null) {
|
||||||
throw new IOException(getBaseDir() + " does not exist.");
|
throw new IOException(getBaseDir() + " does not exist.");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -643,7 +646,7 @@ void generateInputs(int[] opsPerThread) throws IOException {
|
|||||||
long executeOp(int daemonId, int inputIdx, String ignore)
|
long executeOp(int daemonId, int inputIdx, String ignore)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
long start = System.currentTimeMillis();
|
long start = System.currentTimeMillis();
|
||||||
nameNode.getBlockLocations(fileNames[daemonId][inputIdx], 0L, BLOCK_SIZE);
|
nameNodeProto.getBlockLocations(fileNames[daemonId][inputIdx], 0L, BLOCK_SIZE);
|
||||||
long end = System.currentTimeMillis();
|
long end = System.currentTimeMillis();
|
||||||
return end-start;
|
return end-start;
|
||||||
}
|
}
|
||||||
@ -671,7 +674,7 @@ String getOpName() {
|
|||||||
long executeOp(int daemonId, int inputIdx, String ignore)
|
long executeOp(int daemonId, int inputIdx, String ignore)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
long start = System.currentTimeMillis();
|
long start = System.currentTimeMillis();
|
||||||
nameNode.delete(fileNames[daemonId][inputIdx], false);
|
nameNodeProto.delete(fileNames[daemonId][inputIdx], false);
|
||||||
long end = System.currentTimeMillis();
|
long end = System.currentTimeMillis();
|
||||||
return end-start;
|
return end-start;
|
||||||
}
|
}
|
||||||
@ -699,7 +702,7 @@ String getOpName() {
|
|||||||
long executeOp(int daemonId, int inputIdx, String ignore)
|
long executeOp(int daemonId, int inputIdx, String ignore)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
long start = System.currentTimeMillis();
|
long start = System.currentTimeMillis();
|
||||||
nameNode.getFileInfo(fileNames[daemonId][inputIdx]);
|
nameNodeProto.getFileInfo(fileNames[daemonId][inputIdx]);
|
||||||
long end = System.currentTimeMillis();
|
long end = System.currentTimeMillis();
|
||||||
return end-start;
|
return end-start;
|
||||||
}
|
}
|
||||||
@ -741,7 +744,7 @@ void generateInputs(int[] opsPerThread) throws IOException {
|
|||||||
long executeOp(int daemonId, int inputIdx, String ignore)
|
long executeOp(int daemonId, int inputIdx, String ignore)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
long start = System.currentTimeMillis();
|
long start = System.currentTimeMillis();
|
||||||
nameNode.rename(fileNames[daemonId][inputIdx],
|
nameNodeProto.rename(fileNames[daemonId][inputIdx],
|
||||||
destNames[daemonId][inputIdx]);
|
destNames[daemonId][inputIdx]);
|
||||||
long end = System.currentTimeMillis();
|
long end = System.currentTimeMillis();
|
||||||
return end-start;
|
return end-start;
|
||||||
@ -788,11 +791,11 @@ String getName() {
|
|||||||
|
|
||||||
void register() throws IOException {
|
void register() throws IOException {
|
||||||
// get versions from the namenode
|
// get versions from the namenode
|
||||||
nsInfo = nameNode.versionRequest();
|
nsInfo = nameNodeProto.versionRequest();
|
||||||
dnRegistration.setStorageInfo(new DataStorage(nsInfo, ""));
|
dnRegistration.setStorageInfo(new DataStorage(nsInfo, ""));
|
||||||
DataNode.setNewStorageID(dnRegistration);
|
DataNode.setNewStorageID(dnRegistration);
|
||||||
// register datanode
|
// register datanode
|
||||||
dnRegistration = nameNode.registerDatanode(dnRegistration);
|
dnRegistration = nameNodeProto.registerDatanode(dnRegistration);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -802,7 +805,7 @@ void register() throws IOException {
|
|||||||
void sendHeartbeat() throws IOException {
|
void sendHeartbeat() throws IOException {
|
||||||
// register datanode
|
// register datanode
|
||||||
// TODO:FEDERATION currently a single block pool is supported
|
// TODO:FEDERATION currently a single block pool is supported
|
||||||
DatanodeCommand[] cmds = nameNode.sendHeartbeat(dnRegistration,
|
DatanodeCommand[] cmds = nameNodeProto.sendHeartbeat(dnRegistration,
|
||||||
DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0, 0, 0);
|
DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0, 0, 0);
|
||||||
if(cmds != null) {
|
if(cmds != null) {
|
||||||
for (DatanodeCommand cmd : cmds ) {
|
for (DatanodeCommand cmd : cmds ) {
|
||||||
@ -847,7 +850,7 @@ public int compareTo(String name) {
|
|||||||
int replicateBlocks() throws IOException {
|
int replicateBlocks() throws IOException {
|
||||||
// register datanode
|
// register datanode
|
||||||
// TODO:FEDERATION currently a single block pool is supported
|
// TODO:FEDERATION currently a single block pool is supported
|
||||||
DatanodeCommand[] cmds = nameNode.sendHeartbeat(dnRegistration,
|
DatanodeCommand[] cmds = nameNodeProto.sendHeartbeat(dnRegistration,
|
||||||
DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0, 0, 0);
|
DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0, 0, 0);
|
||||||
if (cmds != null) {
|
if (cmds != null) {
|
||||||
for (DatanodeCommand cmd : cmds) {
|
for (DatanodeCommand cmd : cmds) {
|
||||||
@ -878,7 +881,7 @@ private int transferBlocks( Block blocks[],
|
|||||||
receivedDNReg.setStorageInfo(
|
receivedDNReg.setStorageInfo(
|
||||||
new DataStorage(nsInfo, dnInfo.getStorageID()));
|
new DataStorage(nsInfo, dnInfo.getStorageID()));
|
||||||
receivedDNReg.setInfoPort(dnInfo.getInfoPort());
|
receivedDNReg.setInfoPort(dnInfo.getInfoPort());
|
||||||
nameNode.blockReceivedAndDeleted(receivedDNReg, nameNode
|
nameNodeProto.blockReceivedAndDeleted(receivedDNReg, nameNode
|
||||||
.getNamesystem().getBlockPoolId(),
|
.getNamesystem().getBlockPoolId(),
|
||||||
new ReceivedDeletedBlockInfo[] { new ReceivedDeletedBlockInfo(
|
new ReceivedDeletedBlockInfo[] { new ReceivedDeletedBlockInfo(
|
||||||
blocks[i], DataNode.EMPTY_DEL_HINT) });
|
blocks[i], DataNode.EMPTY_DEL_HINT) });
|
||||||
@ -969,14 +972,14 @@ void generateInputs(int[] ignore) throws IOException {
|
|||||||
FileNameGenerator nameGenerator;
|
FileNameGenerator nameGenerator;
|
||||||
nameGenerator = new FileNameGenerator(getBaseDir(), 100);
|
nameGenerator = new FileNameGenerator(getBaseDir(), 100);
|
||||||
String clientName = getClientName(007);
|
String clientName = getClientName(007);
|
||||||
nameNode.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
nameNodeProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
||||||
for(int idx=0; idx < nrFiles; idx++) {
|
for(int idx=0; idx < nrFiles; idx++) {
|
||||||
String fileName = nameGenerator.getNextFileName("ThroughputBench");
|
String fileName = nameGenerator.getNextFileName("ThroughputBench");
|
||||||
nameNode.create(fileName, FsPermission.getDefault(), clientName,
|
nameNodeProto.create(fileName, FsPermission.getDefault(), clientName,
|
||||||
new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)), true, replication,
|
new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)), true, replication,
|
||||||
BLOCK_SIZE);
|
BLOCK_SIZE);
|
||||||
ExtendedBlock lastBlock = addBlocks(fileName, clientName);
|
ExtendedBlock lastBlock = addBlocks(fileName, clientName);
|
||||||
nameNode.complete(fileName, clientName, lastBlock);
|
nameNodeProto.complete(fileName, clientName, lastBlock);
|
||||||
}
|
}
|
||||||
// prepare block reports
|
// prepare block reports
|
||||||
for(int idx=0; idx < nrDatanodes; idx++) {
|
for(int idx=0; idx < nrDatanodes; idx++) {
|
||||||
@ -988,12 +991,12 @@ private ExtendedBlock addBlocks(String fileName, String clientName)
|
|||||||
throws IOException {
|
throws IOException {
|
||||||
ExtendedBlock prevBlock = null;
|
ExtendedBlock prevBlock = null;
|
||||||
for(int jdx = 0; jdx < blocksPerFile; jdx++) {
|
for(int jdx = 0; jdx < blocksPerFile; jdx++) {
|
||||||
LocatedBlock loc = nameNode.addBlock(fileName, clientName, prevBlock, null);
|
LocatedBlock loc = nameNodeProto.addBlock(fileName, clientName, prevBlock, null);
|
||||||
prevBlock = loc.getBlock();
|
prevBlock = loc.getBlock();
|
||||||
for(DatanodeInfo dnInfo : loc.getLocations()) {
|
for(DatanodeInfo dnInfo : loc.getLocations()) {
|
||||||
int dnIdx = Arrays.binarySearch(datanodes, dnInfo.getName());
|
int dnIdx = Arrays.binarySearch(datanodes, dnInfo.getName());
|
||||||
datanodes[dnIdx].addBlock(loc.getBlock().getLocalBlock());
|
datanodes[dnIdx].addBlock(loc.getBlock().getLocalBlock());
|
||||||
nameNode.blockReceivedAndDeleted(datanodes[dnIdx].dnRegistration, loc
|
nameNodeProto.blockReceivedAndDeleted(datanodes[dnIdx].dnRegistration, loc
|
||||||
.getBlock().getBlockPoolId(),
|
.getBlock().getBlockPoolId(),
|
||||||
new ReceivedDeletedBlockInfo[] { new ReceivedDeletedBlockInfo(loc
|
new ReceivedDeletedBlockInfo[] { new ReceivedDeletedBlockInfo(loc
|
||||||
.getBlock().getLocalBlock(), "") });
|
.getBlock().getLocalBlock(), "") });
|
||||||
@ -1013,7 +1016,7 @@ long executeOp(int daemonId, int inputIdx, String ignore) throws IOException {
|
|||||||
assert daemonId < numThreads : "Wrong daemonId.";
|
assert daemonId < numThreads : "Wrong daemonId.";
|
||||||
TinyDatanode dn = datanodes[daemonId];
|
TinyDatanode dn = datanodes[daemonId];
|
||||||
long start = System.currentTimeMillis();
|
long start = System.currentTimeMillis();
|
||||||
nameNode.blockReport(dn.dnRegistration, nameNode.getNamesystem()
|
nameNodeProto.blockReport(dn.dnRegistration, nameNode.getNamesystem()
|
||||||
.getBlockPoolId(), dn.getBlockReportList());
|
.getBlockPoolId(), dn.getBlockReportList());
|
||||||
long end = System.currentTimeMillis();
|
long end = System.currentTimeMillis();
|
||||||
return end-start;
|
return end-start;
|
||||||
@ -1146,7 +1149,7 @@ private void decommissionNodes() throws IOException {
|
|||||||
LOG.info("Datanode " + dn.getName() + " is decommissioned.");
|
LOG.info("Datanode " + dn.getName() + " is decommissioned.");
|
||||||
}
|
}
|
||||||
excludeFile.close();
|
excludeFile.close();
|
||||||
nameNode.refreshNodes();
|
nameNodeProto.refreshNodes();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -1160,8 +1163,8 @@ long executeOp(int daemonId, int inputIdx, String ignore) throws IOException {
|
|||||||
assert daemonId < numThreads : "Wrong daemonId.";
|
assert daemonId < numThreads : "Wrong daemonId.";
|
||||||
long start = System.currentTimeMillis();
|
long start = System.currentTimeMillis();
|
||||||
// compute data-node work
|
// compute data-node work
|
||||||
int work = BlockManagerTestUtil.getComputedDatanodeWork(nameNode
|
int work = BlockManagerTestUtil.getComputedDatanodeWork(
|
||||||
.getNamesystem().getBlockManager());
|
nameNode.getNamesystem().getBlockManager());
|
||||||
long end = System.currentTimeMillis();
|
long end = System.currentTimeMillis();
|
||||||
numPendingBlocks += work;
|
numPendingBlocks += work;
|
||||||
if(work == 0)
|
if(work == 0)
|
||||||
|
@ -47,14 +47,6 @@ public static LocatedBlocks getBlockLocations(NameNode namenode,
|
|||||||
src, offset, length, false, true);
|
src, offset, length, false, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the internal RPC server instance.
|
|
||||||
* @return rpc server
|
|
||||||
*/
|
|
||||||
public static Server getRpcServer(NameNode namenode) {
|
|
||||||
return namenode.server;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static DelegationTokenSecretManager getDtSecretManager(
|
public static DelegationTokenSecretManager getDtSecretManager(
|
||||||
final FSNamesystem ns) {
|
final FSNamesystem ns) {
|
||||||
return ns.getDelegationTokenSecretManager();
|
return ns.getDelegationTokenSecretManager();
|
||||||
|
@ -239,10 +239,10 @@ public Object run() throws IOException {
|
|||||||
LOG.info("Innocuous exception", e);
|
LOG.info("Innocuous exception", e);
|
||||||
}
|
}
|
||||||
locatedBlocks = DFSClientAdapter.callGetBlockLocations(
|
locatedBlocks = DFSClientAdapter.callGetBlockLocations(
|
||||||
cluster.getNameNode(), filePath, 0L, bytes.length);
|
cluster.getNameNodeRpc(), filePath, 0L, bytes.length);
|
||||||
} while (locatedBlocks.isUnderConstruction());
|
} while (locatedBlocks.isUnderConstruction());
|
||||||
|
|
||||||
// Force a roll so we get an OP_END_LOG_SEGMENT txn
|
// Force a roll so we get an OP_END_LOG_SEGMENT txn
|
||||||
return cluster.getNameNode().rollEditLog();
|
return cluster.getNameNodeRpc().rollEditLog();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -34,6 +34,7 @@
|
|||||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
||||||
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
|
import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.apache.log4j.Level;
|
import org.apache.log4j.Level;
|
||||||
@ -128,12 +129,13 @@ public void testBackupNodeTailsEdits() throws Exception {
|
|||||||
fileSys = cluster.getFileSystem();
|
fileSys = cluster.getFileSystem();
|
||||||
backup = startBackupNode(conf, StartupOption.BACKUP, 1);
|
backup = startBackupNode(conf, StartupOption.BACKUP, 1);
|
||||||
|
|
||||||
BackupImage bnImage = backup.getBNImage();
|
BackupImage bnImage = (BackupImage) backup.getFSImage();
|
||||||
testBNInSync(cluster, backup, 1);
|
testBNInSync(cluster, backup, 1);
|
||||||
|
|
||||||
// Force a roll -- BN should roll with NN.
|
// Force a roll -- BN should roll with NN.
|
||||||
NameNode nn = cluster.getNameNode();
|
NameNode nn = cluster.getNameNode();
|
||||||
nn.rollEditLog();
|
NamenodeProtocols nnRpc = nn.getRpcServer();
|
||||||
|
nnRpc.rollEditLog();
|
||||||
assertEquals(bnImage.getEditLog().getCurSegmentTxId(),
|
assertEquals(bnImage.getEditLog().getCurSegmentTxId(),
|
||||||
nn.getFSImage().getEditLog().getCurSegmentTxId());
|
nn.getFSImage().getEditLog().getCurSegmentTxId());
|
||||||
|
|
||||||
@ -207,7 +209,9 @@ public Boolean get() {
|
|||||||
LOG.info("Checking for " + src + " on BN");
|
LOG.info("Checking for " + src + " on BN");
|
||||||
try {
|
try {
|
||||||
boolean hasFile = backup.getNamesystem().getFileInfo(src, false) != null;
|
boolean hasFile = backup.getNamesystem().getFileInfo(src, false) != null;
|
||||||
boolean txnIdMatch = backup.getTransactionID() == nn.getTransactionID();
|
boolean txnIdMatch =
|
||||||
|
backup.getRpcServer().getTransactionID() ==
|
||||||
|
nn.getRpcServer().getTransactionID();
|
||||||
return hasFile && txnIdMatch;
|
return hasFile && txnIdMatch;
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
throw new RuntimeException(e);
|
throw new RuntimeException(e);
|
||||||
@ -264,7 +268,7 @@ void testCheckpoint(StartupOption op) throws Exception {
|
|||||||
//
|
//
|
||||||
// Take a checkpoint
|
// Take a checkpoint
|
||||||
//
|
//
|
||||||
long txid = cluster.getNameNode().getTransactionID();
|
long txid = cluster.getNameNodeRpc().getTransactionID();
|
||||||
backup = startBackupNode(conf, op, 1);
|
backup = startBackupNode(conf, op, 1);
|
||||||
waitCheckpointDone(cluster, backup, txid);
|
waitCheckpointDone(cluster, backup, txid);
|
||||||
} catch(IOException e) {
|
} catch(IOException e) {
|
||||||
@ -300,18 +304,18 @@ void testCheckpoint(StartupOption op) throws Exception {
|
|||||||
// Take a checkpoint
|
// Take a checkpoint
|
||||||
//
|
//
|
||||||
backup = startBackupNode(conf, op, 1);
|
backup = startBackupNode(conf, op, 1);
|
||||||
long txid = cluster.getNameNode().getTransactionID();
|
long txid = cluster.getNameNodeRpc().getTransactionID();
|
||||||
waitCheckpointDone(cluster, backup, txid);
|
waitCheckpointDone(cluster, backup, txid);
|
||||||
|
|
||||||
for (int i = 0; i < 10; i++) {
|
for (int i = 0; i < 10; i++) {
|
||||||
fileSys.mkdirs(new Path("file_" + i));
|
fileSys.mkdirs(new Path("file_" + i));
|
||||||
}
|
}
|
||||||
|
|
||||||
txid = cluster.getNameNode().getTransactionID();
|
txid = cluster.getNameNodeRpc().getTransactionID();
|
||||||
backup.doCheckpoint();
|
backup.doCheckpoint();
|
||||||
waitCheckpointDone(cluster, backup, txid);
|
waitCheckpointDone(cluster, backup, txid);
|
||||||
|
|
||||||
txid = cluster.getNameNode().getTransactionID();
|
txid = cluster.getNameNodeRpc().getTransactionID();
|
||||||
backup.doCheckpoint();
|
backup.doCheckpoint();
|
||||||
waitCheckpointDone(cluster, backup, txid);
|
waitCheckpointDone(cluster, backup, txid);
|
||||||
|
|
||||||
|
@ -38,6 +38,7 @@
|
|||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
||||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
@ -152,7 +153,7 @@ public void testBlockCreation() throws IOException {
|
|||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testGetBlockLocations() throws IOException {
|
public void testGetBlockLocations() throws IOException {
|
||||||
final NameNode namenode = cluster.getNameNode();
|
final NamenodeProtocols namenode = cluster.getNameNodeRpc();
|
||||||
final Path p = new Path(BASE_DIR, "file2.dat");
|
final Path p = new Path(BASE_DIR, "file2.dat");
|
||||||
final String src = p.toString();
|
final String src = p.toString();
|
||||||
final FSDataOutputStream out = TestFileCreation.createFile(hdfs, p, 3);
|
final FSDataOutputStream out = TestFileCreation.createFile(hdfs, p, 3);
|
||||||
|
@ -52,6 +52,7 @@
|
|||||||
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode.CheckpointStorage;
|
import org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode.CheckpointStorage;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
|
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
|
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
|
||||||
import org.apache.hadoop.hdfs.tools.DFSAdmin;
|
import org.apache.hadoop.hdfs.tools.DFSAdmin;
|
||||||
@ -982,11 +983,12 @@ public void testCheckpointSignature() throws IOException {
|
|||||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
|
||||||
.format(true).build();
|
.format(true).build();
|
||||||
NameNode nn = cluster.getNameNode();
|
NameNode nn = cluster.getNameNode();
|
||||||
|
NamenodeProtocols nnRpc = nn.getRpcServer();
|
||||||
|
|
||||||
SecondaryNameNode secondary = startSecondaryNameNode(conf);
|
SecondaryNameNode secondary = startSecondaryNameNode(conf);
|
||||||
// prepare checkpoint image
|
// prepare checkpoint image
|
||||||
secondary.doCheckpoint();
|
secondary.doCheckpoint();
|
||||||
CheckpointSignature sig = nn.rollEditLog();
|
CheckpointSignature sig = nnRpc.rollEditLog();
|
||||||
// manipulate the CheckpointSignature fields
|
// manipulate the CheckpointSignature fields
|
||||||
sig.setBlockpoolID("somerandomebpid");
|
sig.setBlockpoolID("somerandomebpid");
|
||||||
sig.clusterID = "somerandomcid";
|
sig.clusterID = "somerandomcid";
|
||||||
@ -1073,8 +1075,10 @@ public void testMultipleSecondaryNamenodes() throws IOException {
|
|||||||
.nameNodePort(9928).build();
|
.nameNodePort(9928).build();
|
||||||
Configuration snConf1 = new HdfsConfiguration(cluster.getConfiguration(0));
|
Configuration snConf1 = new HdfsConfiguration(cluster.getConfiguration(0));
|
||||||
Configuration snConf2 = new HdfsConfiguration(cluster.getConfiguration(1));
|
Configuration snConf2 = new HdfsConfiguration(cluster.getConfiguration(1));
|
||||||
InetSocketAddress nn1RpcAddress = cluster.getNameNode(0).rpcAddress;
|
InetSocketAddress nn1RpcAddress =
|
||||||
InetSocketAddress nn2RpcAddress = cluster.getNameNode(1).rpcAddress;
|
cluster.getNameNode(0).getNameNodeAddress();
|
||||||
|
InetSocketAddress nn2RpcAddress =
|
||||||
|
cluster.getNameNode(1).getNameNodeAddress();
|
||||||
String nn1 = nn1RpcAddress.getHostName() + ":" + nn1RpcAddress.getPort();
|
String nn1 = nn1RpcAddress.getHostName() + ":" + nn1RpcAddress.getPort();
|
||||||
String nn2 = nn2RpcAddress.getHostName() + ":" + nn2RpcAddress.getPort();
|
String nn2 = nn2RpcAddress.getHostName() + ":" + nn2RpcAddress.getPort();
|
||||||
|
|
||||||
@ -1444,9 +1448,9 @@ public void testNamespaceVerifiedOnFileTransfer() throws IOException {
|
|||||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
|
||||||
.format(true).build();
|
.format(true).build();
|
||||||
|
|
||||||
NameNode nn = cluster.getNameNode();
|
NamenodeProtocols nn = cluster.getNameNodeRpc();
|
||||||
String fsName = NameNode.getHostPortString(nn.getHttpAddress());
|
String fsName = NameNode.getHostPortString(
|
||||||
|
cluster.getNameNode().getHttpAddress());
|
||||||
|
|
||||||
// Make a finalized log on the server side.
|
// Make a finalized log on the server side.
|
||||||
nn.rollEditLog();
|
nn.rollEditLog();
|
||||||
@ -1515,8 +1519,8 @@ public void testCheckpointWithFailedStorageDir() throws Exception {
|
|||||||
|
|
||||||
// Now primary NN experiences failure of a volume -- fake by
|
// Now primary NN experiences failure of a volume -- fake by
|
||||||
// setting its current dir to a-x permissions
|
// setting its current dir to a-x permissions
|
||||||
NameNode nn = cluster.getNameNode();
|
NamenodeProtocols nn = cluster.getNameNodeRpc();
|
||||||
NNStorage storage = nn.getFSImage().getStorage();
|
NNStorage storage = cluster.getNameNode().getFSImage().getStorage();
|
||||||
StorageDirectory sd0 = storage.getStorageDir(0);
|
StorageDirectory sd0 = storage.getStorageDir(0);
|
||||||
StorageDirectory sd1 = storage.getStorageDir(1);
|
StorageDirectory sd1 = storage.getStorageDir(1);
|
||||||
|
|
||||||
@ -1590,8 +1594,8 @@ public void testCheckpointWithSeparateDirsAfterNameFails() throws Exception {
|
|||||||
|
|
||||||
// Now primary NN experiences failure of its only name dir -- fake by
|
// Now primary NN experiences failure of its only name dir -- fake by
|
||||||
// setting its current dir to a-x permissions
|
// setting its current dir to a-x permissions
|
||||||
NameNode nn = cluster.getNameNode();
|
NamenodeProtocols nn = cluster.getNameNodeRpc();
|
||||||
NNStorage storage = nn.getFSImage().getStorage();
|
NNStorage storage = cluster.getNameNode().getFSImage().getStorage();
|
||||||
StorageDirectory sd0 = storage.getStorageDir(0);
|
StorageDirectory sd0 = storage.getStorageDir(0);
|
||||||
assertEquals(NameNodeDirType.IMAGE, sd0.getStorageDirType());
|
assertEquals(NameNodeDirType.IMAGE, sd0.getStorageDirType());
|
||||||
currentDir = sd0.getCurrentDir();
|
currentDir = sd0.getCurrentDir();
|
||||||
@ -1704,7 +1708,7 @@ public void testSecondaryHasVeryOutOfDateImage() throws IOException {
|
|||||||
secondary.doCheckpoint();
|
secondary.doCheckpoint();
|
||||||
|
|
||||||
// Now primary NN saves namespace 3 times
|
// Now primary NN saves namespace 3 times
|
||||||
NameNode nn = cluster.getNameNode();
|
NamenodeProtocols nn = cluster.getNameNodeRpc();
|
||||||
nn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
nn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
||||||
for (int i = 0; i < 3; i++) {
|
for (int i = 0; i < 3; i++) {
|
||||||
nn.saveNamespace();
|
nn.saveNamespace();
|
||||||
|
@ -103,7 +103,7 @@ public void testDeadDatanode() throws Exception {
|
|||||||
dn.shutdown();
|
dn.shutdown();
|
||||||
waitForDatanodeState(reg.getStorageID(), false, 20000);
|
waitForDatanodeState(reg.getStorageID(), false, 20000);
|
||||||
|
|
||||||
DatanodeProtocol dnp = cluster.getNameNode();
|
DatanodeProtocol dnp = cluster.getNameNodeRpc();
|
||||||
|
|
||||||
ReceivedDeletedBlockInfo[] blocks = { new ReceivedDeletedBlockInfo(
|
ReceivedDeletedBlockInfo[] blocks = { new ReceivedDeletedBlockInfo(
|
||||||
new Block(0), "") };
|
new Block(0), "") };
|
||||||
|
@ -47,6 +47,7 @@
|
|||||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
|
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
|
||||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.apache.hadoop.hdfs.tools.DFSck;
|
import org.apache.hadoop.hdfs.tools.DFSck;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
@ -481,7 +482,7 @@ public void testFsckListCorruptFilesBlocks() throws Exception {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// wait for the namenode to see the corruption
|
// wait for the namenode to see the corruption
|
||||||
final NameNode namenode = cluster.getNameNode();
|
final NamenodeProtocols namenode = cluster.getNameNodeRpc();
|
||||||
CorruptFileBlocks corruptFileBlocks = namenode
|
CorruptFileBlocks corruptFileBlocks = namenode
|
||||||
.listCorruptFileBlocks("/corruptData", null);
|
.listCorruptFileBlocks("/corruptData", null);
|
||||||
int numCorrupt = corruptFileBlocks.getFiles().length;
|
int numCorrupt = corruptFileBlocks.getFiles().length;
|
||||||
|
@ -40,6 +40,7 @@
|
|||||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.apache.hadoop.hdfs.tools.DFSAdmin;
|
import org.apache.hadoop.hdfs.tools.DFSAdmin;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
@ -52,7 +53,7 @@ public class TestHDFSConcat {
|
|||||||
private static final short REPL_FACTOR = 2;
|
private static final short REPL_FACTOR = 2;
|
||||||
|
|
||||||
private MiniDFSCluster cluster;
|
private MiniDFSCluster cluster;
|
||||||
private NameNode nn;
|
private NamenodeProtocols nn;
|
||||||
private DistributedFileSystem dfs;
|
private DistributedFileSystem dfs;
|
||||||
|
|
||||||
private static long blockSize = 512;
|
private static long blockSize = 512;
|
||||||
@ -72,7 +73,7 @@ public void startUpCluster() throws IOException {
|
|||||||
cluster.waitClusterUp();
|
cluster.waitClusterUp();
|
||||||
dfs = (DistributedFileSystem) cluster.getFileSystem();
|
dfs = (DistributedFileSystem) cluster.getFileSystem();
|
||||||
assertNotNull("Failed to get FileSystem", dfs);
|
assertNotNull("Failed to get FileSystem", dfs);
|
||||||
nn = cluster.getNameNode();
|
nn = cluster.getNameNodeRpc();
|
||||||
assertNotNull("Failed to get NameNode", nn);
|
assertNotNull("Failed to get NameNode", nn);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -283,7 +284,7 @@ public void testConcatNotCompleteBlock() throws IOException {
|
|||||||
Path filePath1 = new Path(name1);
|
Path filePath1 = new Path(name1);
|
||||||
DFSTestUtil.createFile(dfs, filePath1, trgFileLen, REPL_FACTOR, 1);
|
DFSTestUtil.createFile(dfs, filePath1, trgFileLen, REPL_FACTOR, 1);
|
||||||
|
|
||||||
HdfsFileStatus fStatus = cluster.getNameNode().getFileInfo(name1);
|
HdfsFileStatus fStatus = nn.getFileInfo(name1);
|
||||||
long fileLen = fStatus.getLen();
|
long fileLen = fStatus.getLen();
|
||||||
assertEquals(fileLen, trgFileLen);
|
assertEquals(fileLen, trgFileLen);
|
||||||
|
|
||||||
@ -293,11 +294,11 @@ public void testConcatNotCompleteBlock() throws IOException {
|
|||||||
stm.readFully(0, byteFile1);
|
stm.readFully(0, byteFile1);
|
||||||
stm.close();
|
stm.close();
|
||||||
|
|
||||||
LocatedBlocks lb1 = cluster.getNameNode().getBlockLocations(name1, 0, trgFileLen);
|
LocatedBlocks lb1 = nn.getBlockLocations(name1, 0, trgFileLen);
|
||||||
|
|
||||||
Path filePath2 = new Path(name2);
|
Path filePath2 = new Path(name2);
|
||||||
DFSTestUtil.createFile(dfs, filePath2, srcFileLen, REPL_FACTOR, 1);
|
DFSTestUtil.createFile(dfs, filePath2, srcFileLen, REPL_FACTOR, 1);
|
||||||
fStatus = cluster.getNameNode().getFileInfo(name2);
|
fStatus = nn.getFileInfo(name2);
|
||||||
fileLen = fStatus.getLen();
|
fileLen = fStatus.getLen();
|
||||||
assertEquals(srcFileLen, fileLen);
|
assertEquals(srcFileLen, fileLen);
|
||||||
|
|
||||||
@ -307,7 +308,7 @@ public void testConcatNotCompleteBlock() throws IOException {
|
|||||||
stm.readFully(0, byteFile2);
|
stm.readFully(0, byteFile2);
|
||||||
stm.close();
|
stm.close();
|
||||||
|
|
||||||
LocatedBlocks lb2 = cluster.getNameNode().getBlockLocations(name2, 0, srcFileLen);
|
LocatedBlocks lb2 = nn.getBlockLocations(name2, 0, srcFileLen);
|
||||||
|
|
||||||
|
|
||||||
System.out.println("trg len="+trgFileLen+"; src len="+srcFileLen);
|
System.out.println("trg len="+trgFileLen+"; src len="+srcFileLen);
|
||||||
@ -316,7 +317,7 @@ public void testConcatNotCompleteBlock() throws IOException {
|
|||||||
dfs.concat(filePath1, new Path [] {filePath2});
|
dfs.concat(filePath1, new Path [] {filePath2});
|
||||||
|
|
||||||
long totalLen = trgFileLen + srcFileLen;
|
long totalLen = trgFileLen + srcFileLen;
|
||||||
fStatus = cluster.getNameNode().getFileInfo(name1);
|
fStatus = nn.getFileInfo(name1);
|
||||||
fileLen = fStatus.getLen();
|
fileLen = fStatus.getLen();
|
||||||
|
|
||||||
// read the resulting file
|
// read the resulting file
|
||||||
@ -325,7 +326,7 @@ public void testConcatNotCompleteBlock() throws IOException {
|
|||||||
stm.readFully(0, byteFileConcat);
|
stm.readFully(0, byteFileConcat);
|
||||||
stm.close();
|
stm.close();
|
||||||
|
|
||||||
LocatedBlocks lbConcat = cluster.getNameNode().getBlockLocations(name1, 0, fileLen);
|
LocatedBlocks lbConcat = nn.getBlockLocations(name1, 0, fileLen);
|
||||||
|
|
||||||
//verifications
|
//verifications
|
||||||
// 1. number of blocks
|
// 1. number of blocks
|
||||||
@ -337,7 +338,7 @@ public void testConcatNotCompleteBlock() throws IOException {
|
|||||||
assertEquals(fileLen, totalLen);
|
assertEquals(fileLen, totalLen);
|
||||||
|
|
||||||
// 3. removal of the src file
|
// 3. removal of the src file
|
||||||
fStatus = cluster.getNameNode().getFileInfo(name2);
|
fStatus = nn.getFileInfo(name2);
|
||||||
assertNull("File "+name2+ "still exists", fStatus); // file shouldn't exist
|
assertNull("File "+name2+ "still exists", fStatus); // file shouldn't exist
|
||||||
|
|
||||||
// 4. content
|
// 4. content
|
||||||
|
@ -147,7 +147,7 @@ public void testListCorruptFileBlocksInSafeMode() throws Exception {
|
|||||||
conf.setFloat(DFSConfigKeys.DFS_NAMENODE_REPL_QUEUE_THRESHOLD_PCT_KEY,
|
conf.setFloat(DFSConfigKeys.DFS_NAMENODE_REPL_QUEUE_THRESHOLD_PCT_KEY,
|
||||||
0f);
|
0f);
|
||||||
cluster = new MiniDFSCluster.Builder(conf).waitSafeMode(false).build();
|
cluster = new MiniDFSCluster.Builder(conf).waitSafeMode(false).build();
|
||||||
cluster.getNameNode().
|
cluster.getNameNodeRpc().
|
||||||
setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
||||||
FileSystem fs = cluster.getFileSystem();
|
FileSystem fs = cluster.getFileSystem();
|
||||||
|
|
||||||
@ -244,7 +244,7 @@ public void testListCorruptFileBlocksInSafeMode() throws Exception {
|
|||||||
cluster.getNameNode().isInSafeMode());
|
cluster.getNameNode().isInSafeMode());
|
||||||
|
|
||||||
// now leave safe mode so that we can clean up
|
// now leave safe mode so that we can clean up
|
||||||
cluster.getNameNode().
|
cluster.getNameNodeRpc().
|
||||||
setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
||||||
|
|
||||||
util.cleanup(fs, "/srcdat10");
|
util.cleanup(fs, "/srcdat10");
|
||||||
|
@ -148,8 +148,8 @@ public void testPurgingWithNameEditsDirAfterFailure()
|
|||||||
|
|
||||||
private static void doSaveNamespace(NameNode nn) throws IOException {
|
private static void doSaveNamespace(NameNode nn) throws IOException {
|
||||||
LOG.info("Saving namespace...");
|
LOG.info("Saving namespace...");
|
||||||
nn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
nn.getRpcServer().setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
||||||
nn.saveNamespace();
|
nn.getRpcServer().saveNamespace();
|
||||||
nn.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
|
nn.getRpcServer().setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -28,6 +28,7 @@
|
|||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
@ -54,7 +55,7 @@ public void tearDown() throws Exception {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDelegationToken() throws IOException, InterruptedException {
|
public void testDelegationToken() throws IOException, InterruptedException {
|
||||||
NameNode nn = cluster.getNameNode();
|
NamenodeProtocols nn = cluster.getNameNodeRpc();
|
||||||
HttpServletRequest request = mock(HttpServletRequest.class);
|
HttpServletRequest request = mock(HttpServletRequest.class);
|
||||||
UserGroupInformation ugi = UserGroupInformation.createRemoteUser("auser");
|
UserGroupInformation ugi = UserGroupInformation.createRemoteUser("auser");
|
||||||
String tokenString = NamenodeJspHelper.getDelegationToken(nn, request,
|
String tokenString = NamenodeJspHelper.getDelegationToken(nn, request,
|
||||||
|
@ -108,7 +108,7 @@ public void testRestartDFS() throws Exception {
|
|||||||
files.cleanup(fs, dir);
|
files.cleanup(fs, dir);
|
||||||
files.createFiles(fs, dir);
|
files.createFiles(fs, dir);
|
||||||
fsn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
fsn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
||||||
cluster.getNameNode().saveNamespace();
|
cluster.getNameNodeRpc().saveNamespace();
|
||||||
final String checkAfterModify = checkImages(fsn, numNamenodeDirs);
|
final String checkAfterModify = checkImages(fsn, numNamenodeDirs);
|
||||||
assertFalse("Modified namespace should change fsimage contents. " +
|
assertFalse("Modified namespace should change fsimage contents. " +
|
||||||
"was: " + checkAfterRestart + " now: " + checkAfterModify,
|
"was: " + checkAfterRestart + " now: " + checkAfterModify,
|
||||||
|
@ -51,6 +51,7 @@
|
|||||||
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.apache.hadoop.hdfs.util.MD5FileUtils;
|
import org.apache.hadoop.hdfs.util.MD5FileUtils;
|
||||||
import org.apache.hadoop.io.MD5Hash;
|
import org.apache.hadoop.io.MD5Hash;
|
||||||
import org.apache.hadoop.util.StringUtils;
|
import org.apache.hadoop.util.StringUtils;
|
||||||
@ -379,9 +380,10 @@ public void testCompression() throws IOException {
|
|||||||
NameNode namenode = new NameNode(conf);
|
NameNode namenode = new NameNode(conf);
|
||||||
namenode.getNamesystem().mkdirs("/test",
|
namenode.getNamesystem().mkdirs("/test",
|
||||||
new PermissionStatus("hairong", null, FsPermission.getDefault()), true);
|
new PermissionStatus("hairong", null, FsPermission.getDefault()), true);
|
||||||
assertTrue(namenode.getFileInfo("/test").isDir());
|
NamenodeProtocols nnRpc = namenode.getRpcServer();
|
||||||
namenode.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
assertTrue(nnRpc.getFileInfo("/test").isDir());
|
||||||
namenode.saveNamespace();
|
nnRpc.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
||||||
|
nnRpc.saveNamespace();
|
||||||
namenode.stop();
|
namenode.stop();
|
||||||
namenode.join();
|
namenode.join();
|
||||||
|
|
||||||
@ -408,9 +410,10 @@ public void testCompression() throws IOException {
|
|||||||
|
|
||||||
private void checkNameSpace(Configuration conf) throws IOException {
|
private void checkNameSpace(Configuration conf) throws IOException {
|
||||||
NameNode namenode = new NameNode(conf);
|
NameNode namenode = new NameNode(conf);
|
||||||
assertTrue(namenode.getFileInfo("/test").isDir());
|
NamenodeProtocols nnRpc = namenode.getRpcServer();
|
||||||
namenode.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
assertTrue(nnRpc.getFileInfo("/test").isDir());
|
||||||
namenode.saveNamespace();
|
nnRpc.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
||||||
|
nnRpc.saveNamespace();
|
||||||
namenode.stop();
|
namenode.stop();
|
||||||
namenode.join();
|
namenode.join();
|
||||||
}
|
}
|
||||||
@ -515,7 +518,7 @@ public void testNNRestart() throws IOException, InterruptedException {
|
|||||||
cluster.waitActive();
|
cluster.waitActive();
|
||||||
|
|
||||||
cluster.restartNameNode();
|
cluster.restartNameNode();
|
||||||
NameNode nn = cluster.getNameNode();
|
NamenodeProtocols nn = cluster.getNameNodeRpc();
|
||||||
assertNotNull(nn);
|
assertNotNull(nn);
|
||||||
Assert.assertTrue(cluster.isDataNodeUp());
|
Assert.assertTrue(cluster.isDataNodeUp());
|
||||||
|
|
||||||
|
@ -339,7 +339,7 @@ public void testMultipleSecondaryCheckpoint() throws IOException {
|
|||||||
|
|
||||||
// Simulate a 2NN beginning a checkpoint, but not finishing. This will
|
// Simulate a 2NN beginning a checkpoint, but not finishing. This will
|
||||||
// cause name1 to be restored.
|
// cause name1 to be restored.
|
||||||
cluster.getNameNode().rollEditLog();
|
cluster.getNameNodeRpc().rollEditLog();
|
||||||
|
|
||||||
printStorages(fsImage);
|
printStorages(fsImage);
|
||||||
|
|
||||||
|
@ -74,9 +74,9 @@ public void testFilesInGetListingOps() throws Exception {
|
|||||||
createFile("/tmp1/t2", 3200, (short)3);
|
createFile("/tmp1/t2", 3200, (short)3);
|
||||||
createFile("/tmp2/t1", 3200, (short)3);
|
createFile("/tmp2/t1", 3200, (short)3);
|
||||||
createFile("/tmp2/t2", 3200, (short)3);
|
createFile("/tmp2/t2", 3200, (short)3);
|
||||||
cluster.getNameNode().getListing("/tmp1", HdfsFileStatus.EMPTY_NAME, false);
|
cluster.getNameNodeRpc().getListing("/tmp1", HdfsFileStatus.EMPTY_NAME, false);
|
||||||
assertCounter("FilesInGetListingOps", 2L, getMetrics(NN_METRICS));
|
assertCounter("FilesInGetListingOps", 2L, getMetrics(NN_METRICS));
|
||||||
cluster.getNameNode().getListing("/tmp2", HdfsFileStatus.EMPTY_NAME, false);
|
cluster.getNameNodeRpc().getListing("/tmp2", HdfsFileStatus.EMPTY_NAME, false);
|
||||||
assertCounter("FilesInGetListingOps", 4L, getMetrics(NN_METRICS));
|
assertCounter("FilesInGetListingOps", 4L, getMetrics(NN_METRICS));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -124,8 +124,8 @@ private File initFsimage() throws IOException {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Write results to the fsimage file
|
// Write results to the fsimage file
|
||||||
cluster.getNameNode().setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
cluster.getNameNodeRpc().setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
||||||
cluster.getNameNode().saveNamespace();
|
cluster.getNameNodeRpc().saveNamespace();
|
||||||
|
|
||||||
// Determine location of fsimage file
|
// Determine location of fsimage file
|
||||||
orig = FSImageTestUtil.findLatestImageFile(
|
orig = FSImageTestUtil.findLatestImageFile(
|
||||||
|
Loading…
Reference in New Issue
Block a user