HDFS-1620. Rename HdfsConstants -> HdfsServerConstants, FSConstants -> HdfsConstants. (Harsh J Chouraria via atm)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1169865 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2011-09-12 18:57:21 +00:00
parent cc9f49a7cb
commit 63c7aec4a4
141 changed files with 504 additions and 501 deletions

View File

@ -682,6 +682,9 @@ Release 0.23.0 - Unreleased
HDFS-1217. Change some NameNode methods from public to package private.
(Laxman via szetszwo)
HDFS-1620. Rename HdfsConstants -> HdfsServerConstants, FSConstants ->
HdfsConstants. (Harsh J Chouraria via atm)
OPTIMIZATIONS
HDFS-1458. Improve checkpoint performance by avoiding unnecessary image

View File

@ -505,7 +505,7 @@
using <code>'bin/hadoop dfsadmin -safemode'</code> command. NameNode front
page shows whether Safemode is on or off. A more detailed
description and configuration is maintained as JavaDoc for
<a href="http://hadoop.apache.org/core/docs/current/api/org/apache/hadoop/dfs/NameNode.html#setSafeMode(org.apache.hadoop.dfs.FSConstants.SafeModeAction)"><code>setSafeMode()</code></a>.
<a href="http://hadoop.apache.org/core/docs/current/api/org/apache/hadoop/dfs/NameNode.html#setSafeMode(org.apache.hadoop.dfs.HdfsConstants.SafeModeAction)"><code>setSafeMode()</code></a>.
</p>
</section> <section> <title> fsck </title>

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@ -70,9 +70,9 @@ public class Hdfs extends AbstractFileSystem {
* @throws IOException
*/
Hdfs(final URI theUri, final Configuration conf) throws IOException, URISyntaxException {
super(theUri, FSConstants.HDFS_URI_SCHEME, true, NameNode.DEFAULT_PORT);
super(theUri, HdfsConstants.HDFS_URI_SCHEME, true, NameNode.DEFAULT_PORT);
if (!theUri.getScheme().equalsIgnoreCase(FSConstants.HDFS_URI_SCHEME)) {
if (!theUri.getScheme().equalsIgnoreCase(HdfsConstants.HDFS_URI_SCHEME)) {
throw new IllegalArgumentException("Passed URI's scheme is not for Hdfs");
}
String host = theUri.getHost();

View File

@ -60,10 +60,10 @@ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
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.FSConstants;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
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.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@ -77,7 +77,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
@ -156,14 +156,14 @@ public class DFSClient implements java.io.Closeable {
DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
confTime = conf.getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
HdfsConstants.WRITE_TIMEOUT);
HdfsServerConstants.WRITE_TIMEOUT);
ioBufferSize = conf.getInt(
CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
bytesPerChecksum = conf.getInt(DFS_BYTES_PER_CHECKSUM_KEY,
DFS_BYTES_PER_CHECKSUM_DEFAULT);
socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
HdfsConstants.READ_TIMEOUT);
HdfsServerConstants.READ_TIMEOUT);
/** dfs.write.packet.size is an internal config variable */
writePacketSize = conf.getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
@ -279,12 +279,12 @@ public class DFSClient implements java.io.Closeable {
*/
int getDatanodeWriteTimeout(int numNodes) {
return (dfsClientConf.confTime > 0) ?
(dfsClientConf.confTime + HdfsConstants.WRITE_TIMEOUT_EXTENSION * numNodes) : 0;
(dfsClientConf.confTime + HdfsServerConstants.WRITE_TIMEOUT_EXTENSION * numNodes) : 0;
}
int getDatanodeReadTimeout(int numNodes) {
return dfsClientConf.socketTimeout > 0 ?
(HdfsConstants.READ_TIMEOUT_EXTENSION * numNodes +
(HdfsServerConstants.READ_TIMEOUT_EXTENSION * numNodes +
dfsClientConf.socketTimeout) : 0;
}
@ -1046,7 +1046,7 @@ public class DFSClient implements java.io.Closeable {
out = new DataOutputStream(
new BufferedOutputStream(NetUtils.getOutputStream(sock),
FSConstants.SMALL_BUFFER_SIZE));
HdfsConstants.SMALL_BUFFER_SIZE));
in = new DataInputStream(NetUtils.getInputStream(sock));
if (LOG.isDebugEnabled()) {
@ -1225,7 +1225,7 @@ public class DFSClient implements java.io.Closeable {
/**
* Enter, leave or get safe mode.
*
* @see ClientProtocol#setSafeMode(FSConstants.SafeModeAction)
* @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction)
*/
public boolean setSafeMode(SafeModeAction action) throws IOException {
return namenode.setSafeMode(action);
@ -1293,7 +1293,7 @@ public class DFSClient implements java.io.Closeable {
}
/**
* @see ClientProtocol#distributedUpgradeProgress(FSConstants.UpgradeAction)
* @see ClientProtocol#distributedUpgradeProgress(HdfsConstants.UpgradeAction)
*/
public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
throws IOException {
@ -1392,10 +1392,10 @@ public class DFSClient implements java.io.Closeable {
void setQuota(String src, long namespaceQuota, long diskspaceQuota)
throws IOException {
// sanity check
if ((namespaceQuota <= 0 && namespaceQuota != FSConstants.QUOTA_DONT_SET &&
namespaceQuota != FSConstants.QUOTA_RESET) ||
(diskspaceQuota <= 0 && diskspaceQuota != FSConstants.QUOTA_DONT_SET &&
diskspaceQuota != FSConstants.QUOTA_RESET)) {
if ((namespaceQuota <= 0 && namespaceQuota != HdfsConstants.QUOTA_DONT_SET &&
namespaceQuota != HdfsConstants.QUOTA_RESET) ||
(diskspaceQuota <= 0 && diskspaceQuota != HdfsConstants.QUOTA_DONT_SET &&
diskspaceQuota != HdfsConstants.QUOTA_RESET)) {
throw new IllegalArgumentException("Invalid values for quota : " +
namespaceQuota + " and " +
diskspaceQuota);

View File

@ -47,7 +47,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@ -166,7 +166,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
this.seqno = HEART_BEAT_SEQNO;
buffer = null;
int packetSize = PacketHeader.PKT_HEADER_LEN + FSConstants.BYTES_IN_INTEGER;
int packetSize = PacketHeader.PKT_HEADER_LEN + HdfsConstants.BYTES_IN_INTEGER;
buf = new byte[packetSize];
checksumStart = dataStart = packetSize;
@ -234,12 +234,12 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
dataStart - checksumLen , checksumLen);
}
int pktLen = FSConstants.BYTES_IN_INTEGER + dataLen + checksumLen;
int pktLen = HdfsConstants.BYTES_IN_INTEGER + dataLen + checksumLen;
//normally dataStart == checksumPos, i.e., offset is zero.
buffer = ByteBuffer.wrap(
buf, dataStart - checksumPos,
PacketHeader.PKT_HEADER_LEN + pktLen - FSConstants.BYTES_IN_INTEGER);
PacketHeader.PKT_HEADER_LEN + pktLen - HdfsConstants.BYTES_IN_INTEGER);
buf = null;
buffer.mark();
@ -849,7 +849,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
final long writeTimeout = dfsClient.getDatanodeWriteTimeout(2);
out = new DataOutputStream(new BufferedOutputStream(
NetUtils.getOutputStream(sock, writeTimeout),
FSConstants.SMALL_BUFFER_SIZE));
HdfsConstants.SMALL_BUFFER_SIZE));
//send the TRANSFER_BLOCK request
new Sender(out).transferBlock(block, blockToken, dfsClient.clientName,
@ -1023,7 +1023,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
//
out = new DataOutputStream(new BufferedOutputStream(
NetUtils.getOutputStream(s, writeTimeout),
FSConstants.SMALL_BUFFER_SIZE));
HdfsConstants.SMALL_BUFFER_SIZE));
assert null == blockReplyStream : "Previous blockReplyStream unclosed";
blockReplyStream = new DataInputStream(NetUtils.getInputStream(s));
@ -1173,7 +1173,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
final int timeout = client.getDatanodeReadTimeout(length);
NetUtils.connect(sock, isa, timeout);
sock.setSoTimeout(timeout);
sock.setSendBufferSize(FSConstants.DEFAULT_DATA_SOCKET_SIZE);
sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
if(DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("Send buf size " + sock.getSendBufferSize());
}

View File

@ -48,7 +48,7 @@ import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
@ -646,7 +646,7 @@ public class DFSUtil {
static ClientProtocol createNamenode(ClientProtocol rpcNamenode)
throws IOException {
RetryPolicy createPolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep(
5, FSConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
5, HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
Map<Class<? extends Exception>,RetryPolicy> remoteExceptionToPolicyMap =
new HashMap<Class<? extends Exception>, RetryPolicy>();

View File

@ -49,9 +49,9 @@ import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
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.FSConstants;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@ -108,7 +108,7 @@ public class DistributedFileSystem extends FileSystem {
InetSocketAddress namenode = NameNode.getAddress(uri.getAuthority());
this.dfs = new DFSClient(namenode, conf, statistics);
this.uri = URI.create(FSConstants.HDFS_URI_SCHEME + "://" + uri.getAuthority());
this.uri = URI.create(HdfsConstants.HDFS_URI_SCHEME + "://" + uri.getAuthority());
this.workingDir = getHomeDirectory();
}
@ -642,9 +642,9 @@ public class DistributedFileSystem extends FileSystem {
* Enter, leave or get safe mode.
*
* @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(
* FSConstants.SafeModeAction)
* HdfsConstants.SafeModeAction)
*/
public boolean setSafeMode(FSConstants.SafeModeAction action)
public boolean setSafeMode(HdfsConstants.SafeModeAction action)
throws IOException {
return dfs.setSafeMode(action);
}

View File

@ -30,7 +30,7 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.StringUtils;
@ -162,7 +162,7 @@ class LeaseRenewer {
/** The time in milliseconds that the map became empty. */
private long emptyTime = Long.MAX_VALUE;
/** A fixed lease renewal time period in milliseconds */
private long renewal = FSConstants.LEASE_SOFTLIMIT_PERIOD/2;
private long renewal = HdfsConstants.LEASE_SOFTLIMIT_PERIOD/2;
/** A daemon for renewing lease */
private Daemon daemon = null;
@ -352,7 +352,7 @@ class LeaseRenewer {
//update renewal time
if (renewal == dfsc.getHdfsTimeout()/2) {
long min = FSConstants.LEASE_SOFTLIMIT_PERIOD;
long min = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
for(DFSClient c : dfsclients) {
if (c.getHdfsTimeout() > 0) {
final long timeout = c.getHdfsTimeout();

View File

@ -40,7 +40,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatus
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.token.Token;
@ -394,7 +394,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
throws IOException {
// in and out will be closed when sock is closed (by the caller)
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
NetUtils.getOutputStream(sock, HdfsConstants.WRITE_TIMEOUT)));
NetUtils.getOutputStream(sock, HdfsServerConstants.WRITE_TIMEOUT)));
new Sender(out).readBlock(block, blockToken, clientName, startOffset, len);
//
@ -486,7 +486,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
void sendReadResult(Socket sock, Status statusCode) {
assert !sentStatusCode : "already sent status code to " + sock;
try {
OutputStream out = NetUtils.getOutputStream(sock, HdfsConstants.WRITE_TIMEOUT);
OutputStream out = NetUtils.getOutputStream(sock, HdfsServerConstants.WRITE_TIMEOUT);
ClientReadStatusProto.newBuilder()
.setStatus(statusCode)

View File

@ -22,7 +22,7 @@ import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
/**

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
@ -578,7 +578,7 @@ public interface ClientProtocol extends VersionedProtocol {
* Return live datanodes if type is LIVE; dead datanodes if type is DEAD;
* otherwise all datanodes if type is ALL.
*/
public DatanodeInfo[] getDatanodeReport(FSConstants.DatanodeReportType type)
public DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type)
throws IOException;
/**
@ -601,7 +601,7 @@ public interface ClientProtocol extends VersionedProtocol {
* <p>
* Safe mode is entered automatically at name node startup.
* Safe mode can also be entered manually using
* {@link #setSafeMode(FSConstants.SafeModeAction) setSafeMode(SafeModeAction.SAFEMODE_GET)}.
* {@link #setSafeMode(HdfsConstants.SafeModeAction) setSafeMode(SafeModeAction.SAFEMODE_GET)}.
* <p>
* At startup the name node accepts data node reports collecting
* information about block locations.
@ -617,11 +617,11 @@ public interface ClientProtocol extends VersionedProtocol {
* Then the name node leaves safe mode.
* <p>
* If safe mode is turned on manually using
* {@link #setSafeMode(FSConstants.SafeModeAction) setSafeMode(SafeModeAction.SAFEMODE_ENTER)}
* {@link #setSafeMode(HdfsConstants.SafeModeAction) setSafeMode(SafeModeAction.SAFEMODE_ENTER)}
* then the name node stays in safe mode until it is manually turned off
* using {@link #setSafeMode(FSConstants.SafeModeAction) setSafeMode(SafeModeAction.SAFEMODE_LEAVE)}.
* using {@link #setSafeMode(HdfsConstants.SafeModeAction) setSafeMode(SafeModeAction.SAFEMODE_LEAVE)}.
* Current state of the name node can be verified using
* {@link #setSafeMode(FSConstants.SafeModeAction) setSafeMode(SafeModeAction.SAFEMODE_GET)}
* {@link #setSafeMode(HdfsConstants.SafeModeAction) setSafeMode(SafeModeAction.SAFEMODE_GET)}
* <h4>Configuration parameters:</h4>
* <tt>dfs.safemode.threshold.pct</tt> is the threshold parameter.<br>
* <tt>dfs.safemode.extension</tt> is the safe mode extension parameter.<br>
@ -644,7 +644,7 @@ public interface ClientProtocol extends VersionedProtocol {
*
* @throws IOException
*/
public boolean setSafeMode(FSConstants.SafeModeAction action)
public boolean setSafeMode(HdfsConstants.SafeModeAction action)
throws IOException;
/**
@ -685,7 +685,7 @@ public interface ClientProtocol extends VersionedProtocol {
/**
* Report distributed upgrade progress or force current upgrade to proceed.
*
* @param action {@link FSConstants.UpgradeAction} to perform
* @param action {@link HdfsConstants.UpgradeAction} to perform
* @return upgrade status information or null if no upgrades are in progress
* @throws IOException
*/
@ -777,8 +777,8 @@ public interface ClientProtocol extends VersionedProtocol {
* <br><br>
*
* The quota can have three types of values : (1) 0 or more will set
* the quota to that value, (2) {@link FSConstants#QUOTA_DONT_SET} implies
* the quota will not be changed, and (3) {@link FSConstants#QUOTA_RESET}
* the quota to that value, (2) {@link HdfsConstants#QUOTA_DONT_SET} implies
* the quota will not be changed, and (3) {@link HdfsConstants#QUOTA_RESET}
* implies the quota will be reset. Any other value is a runtime error.
*
* @throws AccessControlException permission denied

View File

@ -26,9 +26,9 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
*
************************************/
@InterfaceAudience.Private
public final class FSConstants {
public final class HdfsConstants {
/* Hidden constructor */
private FSConstants() {
private HdfsConstants() {
}
public static int MIN_BLOCKS_FOR_WRITE = 5;

View File

@ -55,15 +55,15 @@ import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
@ -306,13 +306,13 @@ public class Balancer {
DataInputStream in = null;
try {
sock.connect(NetUtils.createSocketAddr(
target.datanode.getName()), HdfsConstants.READ_TIMEOUT);
target.datanode.getName()), HdfsServerConstants.READ_TIMEOUT);
sock.setKeepAlive(true);
out = new DataOutputStream( new BufferedOutputStream(
sock.getOutputStream(), FSConstants.IO_FILE_BUFFER_SIZE));
sock.getOutputStream(), HdfsConstants.IO_FILE_BUFFER_SIZE));
sendRequest(out);
in = new DataInputStream( new BufferedInputStream(
sock.getInputStream(), FSConstants.IO_FILE_BUFFER_SIZE));
sock.getInputStream(), HdfsConstants.IO_FILE_BUFFER_SIZE));
receiveResponse(in);
bytesMoved.inc(block.getNumBytes());
LOG.info( "Moving block " + block.getBlock().getBlockId() +

View File

@ -18,7 +18,7 @@
package org.apache.hadoop.hdfs.server.blockmanagement;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
import org.apache.hadoop.hdfs.util.LightWeightGSet;

View File

@ -22,8 +22,8 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
/**

View File

@ -50,8 +50,8 @@ import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.INode;

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
import org.apache.hadoop.hdfs.server.namenode.FSInodeInfo;
@ -439,7 +439,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
long remaining = node.getRemaining() -
(node.getBlocksScheduled() * blockSize);
// check the remaining capacity of the target machine
if (blockSize* FSConstants.MIN_BLOCKS_FOR_WRITE>remaining) {
if (blockSize* HdfsConstants.MIN_BLOCKS_FOR_WRITE>remaining) {
if(LOG.isDebugEnabled()) {
threadLocalBuilder.get().append(node.toString()).append(": ")
.append("Node ").append(NodeBase.getPath(node))

View File

@ -45,7 +45,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;

View File

@ -29,9 +29,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
************************************/
@InterfaceAudience.Private
public final class HdfsConstants {
public final class HdfsServerConstants {
/* Hidden constructor */
private HdfsConstants() { }
private HdfsServerConstants() { }
/**
* Type of the node

View File

@ -21,7 +21,7 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
/**
* The exception is thrown when external version does not match
@ -34,7 +34,7 @@ public class IncorrectVersionException extends IOException {
private static final long serialVersionUID = 1L;
public IncorrectVersionException(int versionReported, String ofWhat) {
this(versionReported, ofWhat, FSConstants.LAYOUT_VERSION);
this(versionReported, ofWhat, HdfsConstants.LAYOUT_VERSION);
}
public IncorrectVersionException(int versionReported,

View File

@ -166,8 +166,8 @@ public class JspHelper {
try {
s = new Socket();
s.connect(targetAddr, HdfsConstants.READ_TIMEOUT);
s.setSoTimeout(HdfsConstants.READ_TIMEOUT);
s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT);
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
} catch (IOException e) {
deadNodes.add(chosenNode);
s.close();
@ -188,8 +188,8 @@ public class JspHelper {
JspWriter out, Configuration conf) throws IOException {
if (chunkSizeToView == 0) return;
Socket s = new Socket();
s.connect(addr, HdfsConstants.READ_TIMEOUT);
s.setSoTimeout(HdfsConstants.READ_TIMEOUT);
s.connect(addr, HdfsServerConstants.READ_TIMEOUT);
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
long amtToRead = Math.min(chunkSizeToView, blockSize - offsetIntoBlock);

View File

@ -32,11 +32,11 @@ import java.util.Properties;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.util.VersionInfo;
@ -434,10 +434,10 @@ public abstract class Storage extends StorageInfo {
this.lock(); // lock storage if it exists
if (startOpt == HdfsConstants.StartupOption.FORMAT)
if (startOpt == HdfsServerConstants.StartupOption.FORMAT)
return StorageState.NOT_FORMATTED;
if (startOpt != HdfsConstants.StartupOption.IMPORT) {
if (startOpt != HdfsServerConstants.StartupOption.IMPORT) {
storage.checkOldLayoutStorage(this);
}
@ -866,7 +866,7 @@ public abstract class Storage extends StorageInfo {
* @throws IOException
*/
public void writeAll() throws IOException {
this.layoutVersion = FSConstants.LAYOUT_VERSION;
this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
for (Iterator<StorageDirectory> it = storageDirs.iterator(); it.hasNext();) {
writeProperties(it.next());
}
@ -938,7 +938,7 @@ public abstract class Storage extends StorageInfo {
protected void setLayoutVersion(Properties props, StorageDirectory sd)
throws IncorrectVersionException, InconsistentFSStateException {
int lv = Integer.parseInt(getProperty(props, sd, "layoutVersion"));
if (lv < FSConstants.LAYOUT_VERSION) { // future version
if (lv < HdfsConstants.LAYOUT_VERSION) { // future version
throw new IncorrectVersionException(lv, "storage directory "
+ sd.root.getAbsolutePath());
}

View File

@ -21,7 +21,7 @@ import java.io.IOException;
import java.util.SortedSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
/**
@ -69,7 +69,7 @@ public abstract class UpgradeManager {
currentUpgrades = getDistributedUpgrades();
if(currentUpgrades == null) {
// set new upgrade state
setUpgradeState(false, FSConstants.LAYOUT_VERSION);
setUpgradeState(false, HdfsConstants.LAYOUT_VERSION);
return false;
}
Upgradeable curUO = currentUpgrades.first();
@ -85,7 +85,7 @@ public abstract class UpgradeManager {
return false;
}
public abstract HdfsConstants.NodeType getType();
public abstract HdfsServerConstants.NodeType getType();
public abstract boolean startUpgrade() throws IOException;
public abstract void completeUpgrade() throws IOException;
}

View File

@ -22,7 +22,7 @@ import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.util.StringUtils;
/**
@ -40,7 +40,7 @@ public class UpgradeObjectCollection {
static class UOSignature implements Comparable<UOSignature> {
int version;
HdfsConstants.NodeType type;
HdfsServerConstants.NodeType type;
String className;
UOSignature(Upgradeable uo) {
@ -53,7 +53,7 @@ public class UpgradeObjectCollection {
return version;
}
HdfsConstants.NodeType getType() {
HdfsServerConstants.NodeType getType() {
return type;
}
@ -111,13 +111,13 @@ public class UpgradeObjectCollection {
}
public static SortedSet<Upgradeable> getDistributedUpgrades(int versionFrom,
HdfsConstants.NodeType type
HdfsServerConstants.NodeType type
) throws IOException {
assert FSConstants.LAYOUT_VERSION <= versionFrom : "Incorrect version "
+ versionFrom + ". Expected to be <= " + FSConstants.LAYOUT_VERSION;
assert HdfsConstants.LAYOUT_VERSION <= versionFrom : "Incorrect version "
+ versionFrom + ". Expected to be <= " + HdfsConstants.LAYOUT_VERSION;
SortedSet<Upgradeable> upgradeObjects = new TreeSet<Upgradeable>();
for(UOSignature sig : upgradeTable) {
if(sig.getVersion() < FSConstants.LAYOUT_VERSION)
if(sig.getVersion() < HdfsConstants.LAYOUT_VERSION)
continue;
if(sig.getVersion() > versionFrom)
break;

View File

@ -42,7 +42,7 @@ public interface Upgradeable extends Comparable<Upgradeable> {
* Get the type of the software component, which this object is upgrading.
* @return type
*/
HdfsConstants.NodeType getType();
HdfsServerConstants.NodeType getType();
/**
* Description of the upgrade object for displaying.

View File

@ -30,14 +30,14 @@ import java.util.regex.Pattern;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.HardLink;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.util.Daemon;
@ -89,7 +89,7 @@ public class BlockPoolSliceStorage extends Storage {
*/
void recoverTransitionRead(DataNode datanode, NamespaceInfo nsInfo,
Collection<File> dataDirs, StartupOption startOpt) throws IOException {
assert FSConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion()
assert HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion()
: "Block-pool and name-node layout versions must be the same.";
// 1. For each BP data directory analyze the state and
@ -171,7 +171,7 @@ public class BlockPoolSliceStorage extends Storage {
LOG.info("Formatting block pool " + blockpoolID + " directory "
+ bpSdir.getCurrentDir());
bpSdir.clearDirectory(); // create directory
this.layoutVersion = FSConstants.LAYOUT_VERSION;
this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
this.cTime = nsInfo.getCTime();
this.namespaceID = nsInfo.getNamespaceID();
this.blockpoolID = nsInfo.getBlockPoolID();
@ -239,7 +239,7 @@ public class BlockPoolSliceStorage extends Storage {
readProperties(sd);
checkVersionUpgradable(this.layoutVersion);
assert this.layoutVersion >= FSConstants.LAYOUT_VERSION
assert this.layoutVersion >= HdfsConstants.LAYOUT_VERSION
: "Future version is not allowed";
if (getNamespaceID() != nsInfo.getNamespaceID()) {
throw new IOException("Incompatible namespaceIDs in "
@ -253,7 +253,7 @@ public class BlockPoolSliceStorage extends Storage {
+ nsInfo.getBlockPoolID() + "; datanode blockpoolID = "
+ blockpoolID);
}
if (this.layoutVersion == FSConstants.LAYOUT_VERSION
if (this.layoutVersion == HdfsConstants.LAYOUT_VERSION
&& this.cTime == nsInfo.getCTime())
return; // regular startup
@ -261,7 +261,7 @@ public class BlockPoolSliceStorage extends Storage {
UpgradeManagerDatanode um =
datanode.getUpgradeManagerDatanode(nsInfo.getBlockPoolID());
verifyDistributedUpgradeProgress(um, nsInfo);
if (this.layoutVersion > FSConstants.LAYOUT_VERSION
if (this.layoutVersion > HdfsConstants.LAYOUT_VERSION
|| this.cTime < nsInfo.getCTime()) {
doUpgrade(sd, nsInfo); // upgrade
return;
@ -327,7 +327,7 @@ public class BlockPoolSliceStorage extends Storage {
// 3. Create new <SD>/current with block files hardlinks and VERSION
linkAllBlocks(bpTmpDir, bpCurDir);
this.layoutVersion = FSConstants.LAYOUT_VERSION;
this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
assert this.namespaceID == nsInfo.getNamespaceID()
: "Data-node and name-node layout versions must be the same.";
this.cTime = nsInfo.getCTime();
@ -389,7 +389,7 @@ public class BlockPoolSliceStorage extends Storage {
// the namespace state or can be further upgraded to it.
// In another word, we can only roll back when ( storedLV >= software LV)
// && ( DN.previousCTime <= NN.ctime)
if (!(prevInfo.getLayoutVersion() >= FSConstants.LAYOUT_VERSION &&
if (!(prevInfo.getLayoutVersion() >= HdfsConstants.LAYOUT_VERSION &&
prevInfo.getCTime() <= nsInfo.getCTime())) { // cannot rollback
throw new InconsistentFSStateException(bpSd.getRoot(),
"Cannot rollback to a newer state.\nDatanode previous state: LV = "

View File

@ -36,7 +36,7 @@ import org.apache.commons.logging.Log;
import org.apache.hadoop.fs.FSOutputSummer;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
@ -179,7 +179,7 @@ class BlockReceiver implements Closeable {
this.out = streams.dataOut;
this.cout = streams.checksumOut;
this.checksumOut = new DataOutputStream(new BufferedOutputStream(
streams.checksumOut, FSConstants.SMALL_BUFFER_SIZE));
streams.checksumOut, HdfsConstants.SMALL_BUFFER_SIZE));
// write data chunk header if creating a new replica
if (isCreate) {
BlockMetadataHeader.writeHeader(checksumOut, checksum);
@ -398,7 +398,7 @@ class BlockReceiver implements Closeable {
buf.limit(bufRead);
}
while (buf.remaining() < FSConstants.BYTES_IN_INTEGER) {
while (buf.remaining() < HdfsConstants.BYTES_IN_INTEGER) {
if (buf.position() > 0) {
shiftBufData();
}
@ -420,7 +420,7 @@ class BlockReceiver implements Closeable {
// Subtract BYTES_IN_INTEGER since that accounts for the payloadLen that
// we read above.
int pktSize = payloadLen + PacketHeader.PKT_HEADER_LEN
- FSConstants.BYTES_IN_INTEGER;
- HdfsConstants.BYTES_IN_INTEGER;
if (buf.remaining() < pktSize) {
//we need to read more data

View File

@ -32,7 +32,7 @@ import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.io.IOUtils;
@ -155,7 +155,7 @@ class BlockSender implements java.io.Closeable {
if ( !corruptChecksumOk || datanode.data.metaFileExists(block) ) {
checksumIn = new DataInputStream(new BufferedInputStream(datanode.data
.getMetaDataInputStream(block), FSConstants.IO_FILE_BUFFER_SIZE));
.getMetaDataInputStream(block), HdfsConstants.IO_FILE_BUFFER_SIZE));
// read and handle the common header here. For now just a version
BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
@ -472,14 +472,14 @@ class BlockSender implements java.io.Closeable {
streamForSendChunks = baseStream;
// assure a mininum buffer size.
maxChunksPerPacket = (Math.max(FSConstants.IO_FILE_BUFFER_SIZE,
maxChunksPerPacket = (Math.max(HdfsConstants.IO_FILE_BUFFER_SIZE,
MIN_BUFFER_WITH_TRANSFERTO)
+ bytesPerChecksum - 1)/bytesPerChecksum;
// allocate smaller buffer while using transferTo().
pktSize += checksumSize * maxChunksPerPacket;
} else {
maxChunksPerPacket = Math.max(1, (FSConstants.IO_FILE_BUFFER_SIZE
maxChunksPerPacket = Math.max(1, (HdfsConstants.IO_FILE_BUFFER_SIZE
+ bytesPerChecksum - 1) / bytesPerChecksum);
pktSize += (bytesPerChecksum + checksumSize) * maxChunksPerPacket;
}

View File

@ -68,7 +68,7 @@ import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
@ -83,9 +83,9 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.common.Storage;
@ -435,9 +435,9 @@ public class DataNode extends Configured
private void initConfig(Configuration conf) {
this.socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
HdfsConstants.READ_TIMEOUT);
HdfsServerConstants.READ_TIMEOUT);
this.socketWriteTimeout = conf.getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
HdfsConstants.WRITE_TIMEOUT);
HdfsServerConstants.WRITE_TIMEOUT);
/* Based on results on different platforms, we might need set the default
* to false on some of them. */
this.transferToAllowed = conf.getBoolean(
@ -619,7 +619,7 @@ public class DataNode extends Configured
} else {
ss = secureResources.getStreamingSocket();
}
ss.setReceiveBufferSize(FSConstants.DEFAULT_DATA_SOCKET_SIZE);
ss.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
// adjust machine name with the actual port
int tmpPort = ss.getLocalPort();
selfAddr = new InetSocketAddress(ss.getInetAddress().getHostAddress(),
@ -752,9 +752,9 @@ public class DataNode extends Configured
} catch (InterruptedException ie) {}
}
assert FSConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
assert HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
"Data-node and name-node layout versions must be the same."
+ "Expected: "+ FSConstants.LAYOUT_VERSION
+ "Expected: "+ HdfsConstants.LAYOUT_VERSION
+ " actual "+ nsInfo.getLayoutVersion();
return nsInfo;
}
@ -798,7 +798,7 @@ public class DataNode extends Configured
if (simulatedFSDataset) {
initFsDataSet(conf, dataDirs);
bpRegistration.setStorageID(getStorageId()); //same as DN
bpRegistration.storageInfo.layoutVersion = FSConstants.LAYOUT_VERSION;
bpRegistration.storageInfo.layoutVersion = HdfsConstants.LAYOUT_VERSION;
bpRegistration.storageInfo.namespaceID = bpNSInfo.namespaceID;
bpRegistration.storageInfo.clusterID = bpNSInfo.clusterID;
} else {
@ -1134,9 +1134,9 @@ public class DataNode extends Configured
throw new IncorrectVersionException(nsBuildVer, "namenode", stBuildVer);
}
if (FSConstants.LAYOUT_VERSION != bpNSInfo.getLayoutVersion()) {
if (HdfsConstants.LAYOUT_VERSION != bpNSInfo.getLayoutVersion()) {
LOG.warn("Data-node and name-node layout versions must be " +
"the same. Expected: "+ FSConstants.LAYOUT_VERSION +
"the same. Expected: "+ HdfsConstants.LAYOUT_VERSION +
" actual "+ bpNSInfo.getLayoutVersion());
throw new IncorrectVersionException
(bpNSInfo.getLayoutVersion(), "namenode");
@ -1967,10 +1967,10 @@ public class DataNode extends Configured
sock.setSoTimeout(targets.length * socketTimeout);
long writeTimeout = socketWriteTimeout +
HdfsConstants.WRITE_TIMEOUT_EXTENSION * (targets.length-1);
HdfsServerConstants.WRITE_TIMEOUT_EXTENSION * (targets.length-1);
OutputStream baseStream = NetUtils.getOutputStream(sock, writeTimeout);
out = new DataOutputStream(new BufferedOutputStream(baseStream,
FSConstants.SMALL_BUFFER_SIZE));
HdfsConstants.SMALL_BUFFER_SIZE));
blockSender = new BlockSender(b, 0, b.getNumBytes(),
false, false, false, DataNode.this);
DatanodeInfo srcNode = new DatanodeInfo(bpReg);

View File

@ -43,15 +43,15 @@ import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Daemon;
@ -137,8 +137,8 @@ public class DataStorage extends Storage {
// DN storage has been initialized, no need to do anything
return;
}
assert FSConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
"Data-node version " + FSConstants.LAYOUT_VERSION +
assert HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
"Data-node version " + HdfsConstants.LAYOUT_VERSION +
" and name-node layout version " + nsInfo.getLayoutVersion() +
" must be the same.";
@ -268,7 +268,7 @@ public class DataStorage extends Storage {
void format(StorageDirectory sd, NamespaceInfo nsInfo) throws IOException {
sd.clearDirectory(); // create directory
this.layoutVersion = FSConstants.LAYOUT_VERSION;
this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
this.clusterID = nsInfo.getClusterID();
this.namespaceID = nsInfo.getNamespaceID();
this.cTime = 0;
@ -374,7 +374,7 @@ public class DataStorage extends Storage {
}
readProperties(sd);
checkVersionUpgradable(this.layoutVersion);
assert this.layoutVersion >= FSConstants.LAYOUT_VERSION :
assert this.layoutVersion >= HdfsConstants.LAYOUT_VERSION :
"Future version is not allowed";
boolean federationSupported =
@ -397,7 +397,7 @@ public class DataStorage extends Storage {
}
// regular start up
if (this.layoutVersion == FSConstants.LAYOUT_VERSION
if (this.layoutVersion == HdfsConstants.LAYOUT_VERSION
&& this.cTime == nsInfo.getCTime())
return; // regular startup
// verify necessity of a distributed upgrade
@ -406,7 +406,7 @@ public class DataStorage extends Storage {
verifyDistributedUpgradeProgress(um, nsInfo);
// do upgrade
if (this.layoutVersion > FSConstants.LAYOUT_VERSION
if (this.layoutVersion > HdfsConstants.LAYOUT_VERSION
|| this.cTime < nsInfo.getCTime()) {
doUpgrade(sd, nsInfo); // upgrade
return;
@ -482,7 +482,7 @@ public class DataStorage extends Storage {
linkAllBlocks(tmpDir, new File(curBpDir, STORAGE_DIR_CURRENT));
// 4. Write version file under <SD>/current
layoutVersion = FSConstants.LAYOUT_VERSION;
layoutVersion = HdfsConstants.LAYOUT_VERSION;
clusterID = nsInfo.getClusterID();
writeProperties(sd);
@ -542,7 +542,7 @@ public class DataStorage extends Storage {
// We allow rollback to a state, which is either consistent with
// the namespace state or can be further upgraded to it.
if (!(prevInfo.getLayoutVersion() >= FSConstants.LAYOUT_VERSION
if (!(prevInfo.getLayoutVersion() >= HdfsConstants.LAYOUT_VERSION
&& prevInfo.getCTime() <= nsInfo.getCTime())) // cannot rollback
throw new InconsistentFSStateException(sd.getRoot(),
"Cannot rollback to a newer state.\nDatanode previous state: LV = "

View File

@ -41,7 +41,7 @@ import org.apache.commons.logging.Log;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
@ -53,7 +53,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumR
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.MetaDataInputStream;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.io.IOUtils;
@ -86,7 +86,7 @@ class DataXceiver extends Receiver implements Runnable {
public DataXceiver(Socket s, DataNode datanode,
DataXceiverServer dataXceiverServer) throws IOException {
super(new DataInputStream(new BufferedInputStream(
NetUtils.getInputStream(s), FSConstants.SMALL_BUFFER_SIZE)));
NetUtils.getInputStream(s), HdfsConstants.SMALL_BUFFER_SIZE)));
this.s = s;
this.isLocal = s.getInetAddress().equals(s.getLocalAddress());
@ -203,7 +203,7 @@ class DataXceiver extends Receiver implements Runnable {
OutputStream baseStream = NetUtils.getOutputStream(s,
datanode.socketWriteTimeout);
DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
baseStream, FSConstants.SMALL_BUFFER_SIZE));
baseStream, HdfsConstants.SMALL_BUFFER_SIZE));
checkAccess(out, true, block, blockToken,
Op.READ_BLOCK, BlockTokenSecretManager.AccessMode.READ);
@ -329,7 +329,7 @@ class DataXceiver extends Receiver implements Runnable {
final DataOutputStream replyOut = new DataOutputStream(
new BufferedOutputStream(
NetUtils.getOutputStream(s, datanode.socketWriteTimeout),
FSConstants.SMALL_BUFFER_SIZE));
HdfsConstants.SMALL_BUFFER_SIZE));
checkAccess(replyOut, isClient, block, blockToken,
Op.WRITE_BLOCK, BlockTokenSecretManager.AccessMode.WRITE);
@ -364,16 +364,16 @@ class DataXceiver extends Receiver implements Runnable {
mirrorSock = datanode.newSocket();
try {
int timeoutValue = datanode.socketTimeout
+ (HdfsConstants.READ_TIMEOUT_EXTENSION * targets.length);
+ (HdfsServerConstants.READ_TIMEOUT_EXTENSION * targets.length);
int writeTimeout = datanode.socketWriteTimeout +
(HdfsConstants.WRITE_TIMEOUT_EXTENSION * targets.length);
(HdfsServerConstants.WRITE_TIMEOUT_EXTENSION * targets.length);
NetUtils.connect(mirrorSock, mirrorTarget, timeoutValue);
mirrorSock.setSoTimeout(timeoutValue);
mirrorSock.setSendBufferSize(FSConstants.DEFAULT_DATA_SOCKET_SIZE);
mirrorSock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
mirrorOut = new DataOutputStream(
new BufferedOutputStream(
NetUtils.getOutputStream(mirrorSock, writeTimeout),
FSConstants.SMALL_BUFFER_SIZE));
HdfsConstants.SMALL_BUFFER_SIZE));
mirrorIn = new DataInputStream(NetUtils.getInputStream(mirrorSock));
new Sender(mirrorOut).writeBlock(originalBlock, blockToken,
@ -524,7 +524,7 @@ class DataXceiver extends Receiver implements Runnable {
final MetaDataInputStream metadataIn =
datanode.data.getMetaDataInputStream(block);
final DataInputStream checksumIn = new DataInputStream(new BufferedInputStream(
metadataIn, FSConstants.IO_FILE_BUFFER_SIZE));
metadataIn, HdfsConstants.IO_FILE_BUFFER_SIZE));
updateCurrentThreadName("Getting checksum for block " + block);
try {
@ -603,7 +603,7 @@ class DataXceiver extends Receiver implements Runnable {
OutputStream baseStream = NetUtils.getOutputStream(
s, datanode.socketWriteTimeout);
reply = new DataOutputStream(new BufferedOutputStream(
baseStream, FSConstants.SMALL_BUFFER_SIZE));
baseStream, HdfsConstants.SMALL_BUFFER_SIZE));
// send status first
writeResponse(SUCCESS, reply);
@ -682,14 +682,14 @@ class DataXceiver extends Receiver implements Runnable {
OutputStream baseStream = NetUtils.getOutputStream(proxySock,
datanode.socketWriteTimeout);
proxyOut = new DataOutputStream(new BufferedOutputStream(baseStream,
FSConstants.SMALL_BUFFER_SIZE));
HdfsConstants.SMALL_BUFFER_SIZE));
/* send request to the proxy */
new Sender(proxyOut).copyBlock(block, blockToken);
// receive the response from the proxy
proxyReply = new DataInputStream(new BufferedInputStream(
NetUtils.getInputStream(proxySock), FSConstants.IO_FILE_BUFFER_SIZE));
NetUtils.getInputStream(proxySock), HdfsConstants.IO_FILE_BUFFER_SIZE));
BlockOpResponseProto copyResponse = BlockOpResponseProto.parseFrom(
HdfsProtoUtil.vintPrefixed(proxyReply));

View File

@ -30,7 +30,7 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.balancer.Balancer;
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.io.IOUtils;

View File

@ -53,10 +53,10 @@ import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
@ -465,7 +465,7 @@ public class FSDataset implements FSDatasetInterface {
}
checksumIn = new DataInputStream(
new BufferedInputStream(new FileInputStream(metaFile),
FSConstants.IO_FILE_BUFFER_SIZE));
HdfsConstants.IO_FILE_BUFFER_SIZE));
// read and handle the common header here. For now just a version
BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
import java.io.File;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
/**

View File

@ -18,7 +18,7 @@
package org.apache.hadoop.hdfs.server.datanode;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
/**
* This represents block replicas which are stored in DataNode.

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
import java.io.File;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
/** This class represents replicas being written.

View File

@ -23,7 +23,7 @@ import java.io.IOException;
import java.io.RandomAccessFile;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.BlockWriteStreams;
import org.apache.hadoop.io.IOUtils;

View File

@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.server.datanode;
import java.io.File;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
import java.io.File;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
/**

View File

@ -27,7 +27,7 @@ import org.apache.commons.daemon.DaemonContext;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.http.HttpServer;
import org.mortbay.jetty.nio.SelectChannelConnector;
@ -71,7 +71,7 @@ public class SecureDataNodeStarter implements Daemon {
// Obtain secure port for data streaming to datanode
InetSocketAddress socAddr = DataNode.getStreamingAddr(conf);
int socketWriteTimeout = conf.getInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
HdfsConstants.WRITE_TIMEOUT);
HdfsServerConstants.WRITE_TIMEOUT);
ServerSocket ss = (socketWriteTimeout > 0) ?
ServerSocketChannel.open().socket() : new ServerSocket();

View File

@ -19,8 +19,8 @@ package org.apache.hadoop.hdfs.server.datanode;
import java.io.IOException;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.UpgradeManager;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@ -47,8 +47,8 @@ class UpgradeManagerDatanode extends UpgradeManager {
this.bpid = bpid;
}
public HdfsConstants.NodeType getType() {
return HdfsConstants.NodeType.DATA_NODE;
public HdfsServerConstants.NodeType getType() {
return HdfsServerConstants.NodeType.DATA_NODE;
}
synchronized void initializeUpgrade(NamespaceInfo nsInfo) throws IOException {
@ -57,7 +57,7 @@ class UpgradeManagerDatanode extends UpgradeManager {
DataNode.LOG.info("\n Distributed upgrade for DataNode "
+ dataNode.getMachineName()
+ " version " + getUpgradeVersion() + " to current LV "
+ FSConstants.LAYOUT_VERSION + " is initialized.");
+ HdfsConstants.LAYOUT_VERSION + " is initialized.");
UpgradeObjectDatanode curUO = (UpgradeObjectDatanode)currentUpgrades.first();
curUO.setDatanode(dataNode, this.bpid);
upgradeState = curUO.preUpgradeAction(nsInfo);
@ -102,7 +102,7 @@ class UpgradeManagerDatanode extends UpgradeManager {
if(currentUpgrades == null) {
DataNode.LOG.info("\n Distributed upgrade for DataNode version "
+ getUpgradeVersion() + " to current LV "
+ FSConstants.LAYOUT_VERSION + " cannot be started. "
+ HdfsConstants.LAYOUT_VERSION + " cannot be started. "
+ "The upgrade object is not defined.");
return false;
}
@ -115,7 +115,7 @@ class UpgradeManagerDatanode extends UpgradeManager {
DataNode.LOG.info("\n Distributed upgrade for DataNode "
+ dataNode.getMachineName()
+ " version " + getUpgradeVersion() + " to current LV "
+ FSConstants.LAYOUT_VERSION + " is started.");
+ HdfsConstants.LAYOUT_VERSION + " is started.");
return true;
}
@ -130,7 +130,7 @@ class UpgradeManagerDatanode extends UpgradeManager {
throw new IOException(
"Distributed upgrade for DataNode " + dataNode.getMachineName()
+ " version " + getUpgradeVersion() + " to current LV "
+ FSConstants.LAYOUT_VERSION + " cannot be started. "
+ HdfsConstants.LAYOUT_VERSION + " cannot be started. "
+ "The upgrade object is not defined.");
}
@ -145,7 +145,7 @@ class UpgradeManagerDatanode extends UpgradeManager {
DataNode.LOG.info("\n Distributed upgrade for DataNode "
+ dataNode.getMachineName()
+ " version " + getUpgradeVersion() + " to current LV "
+ FSConstants.LAYOUT_VERSION + " is complete.");
+ HdfsConstants.LAYOUT_VERSION + " is complete.");
}
synchronized void shutdownUpgrade() {

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.hdfs.server.datanode;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.UpgradeObject;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@ -36,8 +36,8 @@ public abstract class UpgradeObjectDatanode extends UpgradeObject implements Run
private DataNode dataNode = null;
private String bpid = null;
public HdfsConstants.NodeType getType() {
return HdfsConstants.NodeType.DATA_NODE;
public HdfsServerConstants.NodeType getType() {
return HdfsServerConstants.NodeType.DATA_NODE;
}
protected DataNode getDatanode() {
@ -118,7 +118,7 @@ public abstract class UpgradeObjectDatanode extends UpgradeObject implements Run
if(getUpgradeStatus() < 100) {
DataNode.LOG.info("\n Distributed upgrade for DataNode version "
+ getVersion() + " to current LV "
+ FSConstants.LAYOUT_VERSION + " cannot be completed.");
+ HdfsConstants.LAYOUT_VERSION + " cannot be completed.");
}
// Complete the upgrade by calling the manager method

View File

@ -25,7 +25,7 @@ import java.util.zip.Checksum;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.common.Storage.StorageState;
@ -106,7 +106,7 @@ public class BackupImage extends FSImage {
StorageDirectory sd = it.next();
StorageState curState;
try {
curState = sd.analyzeStorage(HdfsConstants.StartupOption.REGULAR, storage);
curState = sd.analyzeStorage(HdfsServerConstants.StartupOption.REGULAR, storage);
// sd is locked but not opened
switch(curState) {
case NON_EXISTENT:

View File

@ -26,8 +26,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
@ -134,7 +134,7 @@ public class BackupNode extends NameNode implements JournalProtocol {
// Backup node should never do lease recovery,
// therefore lease hard limit should never expire.
namesystem.leaseManager.setLeasePeriod(
FSConstants.LEASE_SOFTLIMIT_PERIOD, Long.MAX_VALUE);
HdfsConstants.LEASE_SOFTLIMIT_PERIOD, Long.MAX_VALUE);
clusterId = nsInfo.getClusterID();
blockPoolId = nsInfo.getBlockPoolID();
@ -356,9 +356,9 @@ public class BackupNode extends NameNode implements JournalProtocol {
LOG.fatal(errorMsg);
throw new IOException(errorMsg);
}
assert FSConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
assert HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
"Active and backup node layout versions must be the same. Expected: "
+ FSConstants.LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion();
+ HdfsConstants.LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion();
return nsInfo;
}

View File

@ -30,7 +30,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;

View File

@ -24,7 +24,7 @@ import java.io.IOException;
import java.io.BufferedInputStream;
import java.io.EOFException;
import java.io.DataInputStream;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
import org.apache.hadoop.io.IOUtils;
@ -143,11 +143,11 @@ class EditLogFileInputStream extends EditLogInputStream {
throw new LogHeaderCorruptException(
"Reached EOF when reading log header");
}
if (logVersion < FSConstants.LAYOUT_VERSION) { // future version
if (logVersion < HdfsConstants.LAYOUT_VERSION) { // future version
throw new LogHeaderCorruptException(
"Unexpected version of the file system log file: "
+ logVersion + ". Current version = "
+ FSConstants.LAYOUT_VERSION + ".");
+ HdfsConstants.LAYOUT_VERSION + ".");
}
assert logVersion <= Storage.LAST_UPGRADABLE_LAYOUT_VERSION :
"Unsupported version " + logVersion;

View File

@ -27,7 +27,7 @@ import java.nio.channels.FileChannel;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.io.IOUtils;
import com.google.common.annotations.VisibleForTesting;
@ -109,7 +109,7 @@ class EditLogFileOutputStream extends EditLogOutputStream {
void create() throws IOException {
fc.truncate(0);
fc.position(0);
doubleBuf.getCurrentBuf().writeInt(FSConstants.LAYOUT_VERSION);
doubleBuf.getCurrentBuf().writeInt(HdfsConstants.LAYOUT_VERSION);
setReadyToFlush();
flush();
}

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.Writer;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.IOUtils;
@ -129,7 +129,7 @@ class EditsDoubleBuffer {
}
public void writeOp(FSEditLogOp op) throws IOException {
if (firstTxId == FSConstants.INVALID_TXID) {
if (firstTxId == HdfsConstants.INVALID_TXID) {
firstTxId = op.txid;
} else {
assert op.txid > firstTxId;
@ -141,7 +141,7 @@ class EditsDoubleBuffer {
@Override
public DataOutputBuffer reset() {
super.reset();
firstTxId = FSConstants.INVALID_TXID;
firstTxId = HdfsConstants.INVALID_TXID;
numTxns = 0;
return this;
}

View File

@ -43,7 +43,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.FSLimitException;
import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
@ -55,8 +55,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.util.ByteArray;
/*************************************************
@ -1876,10 +1876,10 @@ public class FSDirectory implements Closeable {
UnresolvedLinkException {
assert hasWriteLock();
// sanity check
if ((nsQuota < 0 && nsQuota != FSConstants.QUOTA_DONT_SET &&
nsQuota < FSConstants.QUOTA_RESET) ||
(dsQuota < 0 && dsQuota != FSConstants.QUOTA_DONT_SET &&
dsQuota < FSConstants.QUOTA_RESET)) {
if ((nsQuota < 0 && nsQuota != HdfsConstants.QUOTA_DONT_SET &&
nsQuota < HdfsConstants.QUOTA_RESET) ||
(dsQuota < 0 && dsQuota != HdfsConstants.QUOTA_DONT_SET &&
dsQuota < HdfsConstants.QUOTA_RESET)) {
throw new IllegalArgumentException("Illegal value for nsQuota or " +
"dsQuota : " + nsQuota + " and " +
dsQuota);
@ -1893,16 +1893,16 @@ public class FSDirectory implements Closeable {
throw new FileNotFoundException("Directory does not exist: " + srcs);
} else if (!targetNode.isDirectory()) {
throw new FileNotFoundException("Cannot set quota on a file: " + srcs);
} else if (targetNode.isRoot() && nsQuota == FSConstants.QUOTA_RESET) {
} else if (targetNode.isRoot() && nsQuota == HdfsConstants.QUOTA_RESET) {
throw new IllegalArgumentException("Cannot clear namespace quota on root.");
} else { // a directory inode
INodeDirectory dirNode = (INodeDirectory)targetNode;
long oldNsQuota = dirNode.getNsQuota();
long oldDsQuota = dirNode.getDsQuota();
if (nsQuota == FSConstants.QUOTA_DONT_SET) {
if (nsQuota == HdfsConstants.QUOTA_DONT_SET) {
nsQuota = oldNsQuota;
}
if (dsQuota == FSConstants.QUOTA_DONT_SET) {
if (dsQuota == HdfsConstants.QUOTA_DONT_SET) {
dsQuota = oldDsQuota;
}

View File

@ -29,9 +29,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@ -91,7 +91,7 @@ public class FSEditLog {
// the first txid of the log that's currently open for writing.
// If this value is N, we are currently writing to edits_inprogress_N
private long curSegmentTxId = FSConstants.INVALID_TXID;
private long curSegmentTxId = HdfsConstants.INVALID_TXID;
// the time of printing the statistics to the log file.
private long lastPrintTime;
@ -904,7 +904,7 @@ public class FSEditLog {
// synchronized to prevent findbugs warning about inconsistent
// synchronization. This will be JIT-ed out if asserts are
// off.
assert curSegmentTxId == FSConstants.INVALID_TXID || // on format this is no-op
assert curSegmentTxId == HdfsConstants.INVALID_TXID || // on format this is no-op
minTxIdToKeep <= curSegmentTxId :
"cannot purge logs older than txid " + minTxIdToKeep +
" when current segment starts at " + curSegmentTxId;
@ -1078,7 +1078,7 @@ public class FSEditLog {
static class JournalAndStream {
private final JournalManager manager;
private EditLogOutputStream stream;
private long segmentStartsAtTxId = FSConstants.INVALID_TXID;
private long segmentStartsAtTxId = HdfsConstants.INVALID_TXID;
private JournalAndStream(JournalManager manager) {
this.manager = manager;
@ -1110,7 +1110,7 @@ public class FSEditLog {
LOG.error("Unable to abort stream " + stream, ioe);
}
stream = null;
segmentStartsAtTxId = FSConstants.INVALID_TXID;
segmentStartsAtTxId = HdfsConstants.INVALID_TXID;
}
private boolean isActive() {

View File

@ -27,7 +27,7 @@ import java.util.Arrays;
import java.util.EnumMap;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
@ -274,14 +274,14 @@ public class FSEditLogLoader {
SetNSQuotaOp setNSQuotaOp = (SetNSQuotaOp)op;
fsDir.unprotectedSetQuota(setNSQuotaOp.src,
setNSQuotaOp.nsQuota,
FSConstants.QUOTA_DONT_SET);
HdfsConstants.QUOTA_DONT_SET);
break;
}
case OP_CLEAR_NS_QUOTA: {
ClearNSQuotaOp clearNSQuotaOp = (ClearNSQuotaOp)op;
fsDir.unprotectedSetQuota(clearNSQuotaOp.src,
FSConstants.QUOTA_RESET,
FSConstants.QUOTA_DONT_SET);
HdfsConstants.QUOTA_RESET,
HdfsConstants.QUOTA_DONT_SET);
break;
}
@ -435,7 +435,7 @@ public class FSEditLogLoader {
// The editlog must be emptied by restarting the namenode, before proceeding
// with the upgrade.
if (Storage.is203LayoutVersion(logVersion)
&& logVersion != FSConstants.LAYOUT_VERSION) {
&& logVersion != HdfsConstants.LAYOUT_VERSION) {
String msg = "During upgrade failed to load the editlog version "
+ logVersion + " from release 0.20.203. Please go back to the old "
+ " release and restart the namenode. This empties the editlog "

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;

View File

@ -35,7 +35,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@ -44,8 +44,8 @@ import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.common.Storage.StorageState;
import org.apache.hadoop.hdfs.server.common.Util;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.LoadPlan;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
@ -227,11 +227,11 @@ public class FSImage implements Closeable {
}
if (startOpt != StartupOption.UPGRADE
&& layoutVersion < Storage.LAST_PRE_UPGRADE_LAYOUT_VERSION
&& layoutVersion != FSConstants.LAYOUT_VERSION) {
&& layoutVersion != HdfsConstants.LAYOUT_VERSION) {
throw new IOException(
"\nFile system image contains an old layout version "
+ storage.getLayoutVersion() + ".\nAn upgrade to version "
+ FSConstants.LAYOUT_VERSION + " is required.\n"
+ HdfsConstants.LAYOUT_VERSION + " is required.\n"
+ "Please restart NameNode with -upgrade option.");
}
@ -349,7 +349,7 @@ public class FSImage implements Closeable {
long oldCTime = storage.getCTime();
storage.cTime = now(); // generate new cTime for the state
int oldLV = storage.getLayoutVersion();
storage.layoutVersion = FSConstants.LAYOUT_VERSION;
storage.layoutVersion = HdfsConstants.LAYOUT_VERSION;
List<StorageDirectory> errorSDs =
Collections.synchronizedList(new ArrayList<StorageDirectory>());
@ -423,7 +423,7 @@ public class FSImage implements Closeable {
// Directories that don't have previous state do not rollback
boolean canRollback = false;
FSImage prevState = new FSImage(conf, getFSNamesystem());
prevState.getStorage().layoutVersion = FSConstants.LAYOUT_VERSION;
prevState.getStorage().layoutVersion = HdfsConstants.LAYOUT_VERSION;
for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
StorageDirectory sd = it.next();
File prevDir = sd.getPreviousDir();
@ -438,12 +438,12 @@ public class FSImage implements Closeable {
// read and verify consistency of the prev dir
prevState.getStorage().readPreviousVersionProperties(sd);
if (prevState.getLayoutVersion() != FSConstants.LAYOUT_VERSION) {
if (prevState.getLayoutVersion() != HdfsConstants.LAYOUT_VERSION) {
throw new IOException(
"Cannot rollback to storage version " +
prevState.getLayoutVersion() +
" using this version of the NameNode, which uses storage version " +
FSConstants.LAYOUT_VERSION + ". " +
HdfsConstants.LAYOUT_VERSION + ". " +
"Please use the previous version of HDFS to perform the rollback.");
}
canRollback = true;

View File

@ -39,7 +39,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
@ -555,7 +555,7 @@ class FSImageFormat {
DigestOutputStream fos = new DigestOutputStream(fout, digester);
DataOutputStream out = new DataOutputStream(fos);
try {
out.writeInt(FSConstants.LAYOUT_VERSION);
out.writeInt(HdfsConstants.LAYOUT_VERSION);
out.writeInt(sourceNamesystem.getFSImage()
.getStorage().getNamespaceID()); // TODO bad dependency
out.writeLong(fsDir.rootDir.numItemsInTree());
@ -568,7 +568,7 @@ class FSImageFormat {
" using " + compression);
byte[] byteStore = new byte[4*FSConstants.MAX_PATH_LENGTH];
byte[] byteStore = new byte[4*HdfsConstants.MAX_PATH_LENGTH];
ByteBuffer strbuf = ByteBuffer.wrap(byteStore);
// save the root
FSImageSerialization.saveINode2Image(fsDir.rootDir, out);

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;

View File

@ -35,7 +35,7 @@ import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
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.NameNodeFile;
@ -224,7 +224,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
logGroup.planRecovery();
if (expectedTxId != FSConstants.INVALID_TXID && logStartTxId != expectedTxId) {
if (expectedTxId != HdfsConstants.INVALID_TXID && logStartTxId != expectedTxId) {
throw new IOException("Expected next log group would start at txid " +
expectedTxId + " but starts at txid " + logStartTxId);
}
@ -239,7 +239,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
} else {
// the log group was in-progress so we don't know what ID
// the next group should start from.
expectedTxId = FSConstants.INVALID_TXID;
expectedTxId = HdfsConstants.INVALID_TXID;
}
}

View File

@ -78,10 +78,10 @@ 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.FSConstants;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
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.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@ -99,9 +99,9 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
import org.apache.hadoop.hdfs.server.common.Util;
@ -2756,7 +2756,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* not tracked because the name node is not intended to leave safe mode
* automatically in the case.
*
* @see ClientProtocol#setSafeMode(FSConstants.SafeModeAction)
* @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction)
* @see SafeModeMonitor
*/
class SafeModeInfo {

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DatanodeJspHelper;
@ -120,7 +120,7 @@ public class FileChecksumServlets {
new HdfsConfiguration(datanode.getConf());
final int socketTimeout = conf.getInt(
DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY,
HdfsConstants.READ_TIMEOUT);
HdfsServerConstants.READ_TIMEOUT);
final SocketFactory socketFactory = NetUtils.getSocketFactory(conf,
ClientProtocol.class);

View File

@ -29,7 +29,7 @@ import javax.servlet.http.HttpServletResponse;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.security.UserGroupInformation;

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
/**
* I-node for file being written.

View File

@ -32,8 +32,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import static org.apache.hadoop.hdfs.server.common.Util.now;
@ -65,8 +65,8 @@ public class LeaseManager {
private final FSNamesystem fsnamesystem;
private long softLimit = FSConstants.LEASE_SOFTLIMIT_PERIOD;
private long hardLimit = FSConstants.LEASE_HARDLIMIT_PERIOD;
private long softLimit = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
private long hardLimit = HdfsConstants.LEASE_HARDLIMIT_PERIOD;
//
// Used for handling lock-leases
@ -379,7 +379,7 @@ public class LeaseManager {
try {
Thread.sleep(HdfsConstants.NAMENODE_LEASE_RECHECK_INTERVAL);
Thread.sleep(HdfsServerConstants.NAMENODE_LEASE_RECHECK_INTERVAL);
} catch(InterruptedException ie) {
if (LOG.isDebugEnabled()) {
LOG.debug(name + " is interrupted", ie);
@ -409,7 +409,7 @@ public class LeaseManager {
oldest.getPaths().toArray(leasePaths);
for(String p : leasePaths) {
try {
if(fsnamesystem.internalReleaseLease(oldest, p, HdfsConstants.NAMENODE_LEASE_HOLDER)) {
if(fsnamesystem.internalReleaseLease(oldest, p, HdfsServerConstants.NAMENODE_LEASE_HOLDER)) {
LOG.info("Lease recovery for file " + p +
" is complete. File closed.");
removing.add(p);

View File

@ -42,11 +42,11 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.UpgradeManager;
@ -126,7 +126,7 @@ public class NNStorage extends Storage implements Closeable {
* recent fsimage file. This does not include any transactions
* that have since been written to the edit log.
*/
protected long mostRecentCheckpointTxId = FSConstants.INVALID_TXID;
protected long mostRecentCheckpointTxId = HdfsConstants.INVALID_TXID;
/**
* list of failed (and thus removed) storages
@ -501,7 +501,7 @@ public class NNStorage extends Storage implements Closeable {
* Format all available storage directories.
*/
public void format(String clusterId) throws IOException {
this.layoutVersion = FSConstants.LAYOUT_VERSION;
this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
this.namespaceID = newNamespaceID();
this.clusterID = clusterId;
this.blockpoolID = newBlockPoolID();
@ -574,7 +574,7 @@ public class NNStorage extends Storage implements Closeable {
* This should only be used during upgrades.
*/
String getDeprecatedProperty(String prop) {
assert getLayoutVersion() > FSConstants.LAYOUT_VERSION :
assert getLayoutVersion() > HdfsConstants.LAYOUT_VERSION :
"getDeprecatedProperty should only be done when loading " +
"storage from past versions during upgrade.";
return deprecatedProperties.get(prop);
@ -764,7 +764,7 @@ public class NNStorage extends Storage implements Closeable {
if(upgradeManager.getDistributedUpgrades() != null)
throw new IOException("\n Distributed upgrade for NameNode version "
+ upgradeManager.getUpgradeVersion()
+ " to current LV " + FSConstants.LAYOUT_VERSION
+ " to current LV " + HdfsConstants.LAYOUT_VERSION
+ " is required.\n Please restart NameNode"
+ " with -upgrade option.");
}
@ -780,7 +780,7 @@ public class NNStorage extends Storage implements Closeable {
writeAll();
LOG.info("\n Distributed upgrade for NameNode version "
+ upgradeManager.getUpgradeVersion() + " to current LV "
+ FSConstants.LAYOUT_VERSION + " is initialized.");
+ HdfsConstants.LAYOUT_VERSION + " is initialized.");
}
/**

View File

@ -55,12 +55,12 @@ 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.FSConstants;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
import static org.apache.hadoop.hdfs.protocol.FSConstants.MAX_PATH_LENGTH;
import static org.apache.hadoop.hdfs.protocol.FSConstants.MAX_PATH_DEPTH;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
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 static org.apache.hadoop.hdfs.protocol.HdfsConstants.MAX_PATH_LENGTH;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.MAX_PATH_DEPTH;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@ -68,8 +68,8 @@ import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
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.HdfsConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
@ -307,12 +307,12 @@ public class NameNode implements NamenodeProtocols {
"Invalid URI for NameNode address (check %s): %s has no authority.",
FileSystem.FS_DEFAULT_NAME_KEY, filesystemURI.toString()));
}
if (!FSConstants.HDFS_URI_SCHEME.equalsIgnoreCase(
if (!HdfsConstants.HDFS_URI_SCHEME.equalsIgnoreCase(
filesystemURI.getScheme())) {
throw new IllegalArgumentException(String.format(
"Invalid URI for NameNode address (check %s): %s is not of scheme '%s'.",
FileSystem.FS_DEFAULT_NAME_KEY, filesystemURI.toString(),
FSConstants.HDFS_URI_SCHEME));
HdfsConstants.HDFS_URI_SCHEME));
}
return getAddress(authority);
}
@ -320,7 +320,7 @@ public class NameNode implements NamenodeProtocols {
public static URI getUri(InetSocketAddress namenode) {
int port = namenode.getPort();
String portString = port == DEFAULT_PORT ? "" : (":"+port);
return URI.create(FSConstants.HDFS_URI_SCHEME + "://"
return URI.create(HdfsConstants.HDFS_URI_SCHEME + "://"
+ namenode.getHostName()+portString);
}
@ -1279,7 +1279,7 @@ public class NameNode implements NamenodeProtocols {
* @throws IOException
*/
void verifyVersion(int version) throws IOException {
if (version != FSConstants.LAYOUT_VERSION)
if (version != HdfsConstants.LAYOUT_VERSION)
throw new IncorrectVersionException(version, "data node");
}
@ -1573,7 +1573,7 @@ public class NameNode implements NamenodeProtocols {
DFSUtil.setGenericConf(conf, nameserviceId, NAMESERVICE_SPECIFIC_KEYS);
if (conf.get(DFS_NAMENODE_RPC_ADDRESS_KEY) != null) {
URI defaultUri = URI.create(FSConstants.HDFS_URI_SCHEME + "://"
URI defaultUri = URI.create(HdfsConstants.HDFS_URI_SCHEME + "://"
+ conf.get(DFS_NAMENODE_RPC_ADDRESS_KEY));
conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
}

View File

@ -48,7 +48,7 @@ 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.server.blockmanagement.BlockPlacementPolicy;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.NodeBase;
@ -502,8 +502,8 @@ public class NamenodeFsck {
}
try {
s = new Socket();
s.connect(targetAddr, HdfsConstants.READ_TIMEOUT);
s.setSoTimeout(HdfsConstants.READ_TIMEOUT);
s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT);
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
String file = BlockReaderFactory.getFileName(targetAddr, block.getBlockPoolId(),
block.getBlockId());

View File

@ -40,7 +40,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;

View File

@ -45,8 +45,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DFSUtil.ErrorSimulator;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@ -456,7 +456,7 @@ public class SecondaryNameNode implements Runnable {
*/
private String getInfoServer() throws IOException {
URI fsName = FileSystem.getDefaultUri(conf);
if (!FSConstants.HDFS_URI_SCHEME.equalsIgnoreCase(fsName.getScheme())) {
if (!HdfsConstants.HDFS_URI_SCHEME.equalsIgnoreCase(fsName.getScheme())) {
throw new IOException("This is not a DFS");
}
@ -793,7 +793,7 @@ public class SecondaryNameNode implements Runnable {
StorageState curState;
try {
curState = sd.analyzeStorage(HdfsConstants.StartupOption.REGULAR, storage);
curState = sd.analyzeStorage(HdfsServerConstants.StartupOption.REGULAR, storage);
// sd is locked but not opened
switch(curState) {
case NON_EXISTENT:

View File

@ -27,7 +27,7 @@ import java.lang.Math;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
@ -124,7 +124,7 @@ class TransferFsImage {
static void getFileServer(OutputStream outstream, File localfile,
DataTransferThrottler throttler)
throws IOException {
byte buf[] = new byte[FSConstants.IO_FILE_BUFFER_SIZE];
byte buf[] = new byte[HdfsConstants.IO_FILE_BUFFER_SIZE];
FileInputStream infile = null;
try {
infile = new FileInputStream(localfile);
@ -139,7 +139,7 @@ class TransferFsImage {
&& localfile.getAbsolutePath().contains("fsimage")) {
// Test sending image shorter than localfile
long len = localfile.length();
buf = new byte[(int)Math.min(len/2, FSConstants.IO_FILE_BUFFER_SIZE)];
buf = new byte[(int)Math.min(len/2, HdfsConstants.IO_FILE_BUFFER_SIZE)];
// This will read at most half of the image
// and the rest of the image will be sent over the wire
infile.read(buf);
@ -179,7 +179,7 @@ class TransferFsImage {
static MD5Hash getFileClient(String nnHostPort,
String queryString, List<File> localPaths,
NNStorage dstStorage, boolean getChecksum) throws IOException {
byte[] buf = new byte[FSConstants.IO_FILE_BUFFER_SIZE];
byte[] buf = new byte[HdfsConstants.IO_FILE_BUFFER_SIZE];
String proto = UserGroupInformation.isSecurityEnabled() ? "https://" : "http://";
StringBuilder str = new StringBuilder(proto+nnHostPort+"/getimage?");
str.append(queryString);

View File

@ -19,9 +19,9 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
import org.apache.hadoop.hdfs.server.common.UpgradeManager;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
@ -38,8 +38,8 @@ import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
* and updates its status.
*/
class UpgradeManagerNamenode extends UpgradeManager {
public HdfsConstants.NodeType getType() {
return HdfsConstants.NodeType.NAME_NODE;
public HdfsServerConstants.NodeType getType() {
return HdfsServerConstants.NodeType.NAME_NODE;
}
private final FSNamesystem namesystem;
@ -66,7 +66,7 @@ class UpgradeManagerNamenode extends UpgradeManager {
this.broadcastCommand = currentUpgrades.first().startUpgrade();
NameNode.LOG.info("\n Distributed upgrade for NameNode version "
+ getUpgradeVersion() + " to current LV "
+ FSConstants.LAYOUT_VERSION + " is started.");
+ HdfsConstants.LAYOUT_VERSION + " is started.");
return true;
}
@ -75,7 +75,7 @@ class UpgradeManagerNamenode extends UpgradeManager {
if(NameNode.LOG.isDebugEnabled()) {
NameNode.LOG.debug("\n Distributed upgrade for NameNode version "
+ getUpgradeVersion() + " to current LV "
+ FSConstants.LAYOUT_VERSION + " is processing upgrade command: "
+ HdfsConstants.LAYOUT_VERSION + " is processing upgrade command: "
+ command.getAction() + " status = " + getUpgradeStatus() + "%");
}
if(currentUpgrades == null) {
@ -96,7 +96,7 @@ class UpgradeManagerNamenode extends UpgradeManager {
curUO.completeUpgrade();
NameNode.LOG.info("\n Distributed upgrade for NameNode version "
+ curUO.getVersion() + " to current LV "
+ FSConstants.LAYOUT_VERSION + " is complete.");
+ HdfsConstants.LAYOUT_VERSION + " is complete.");
// proceede with the next one
currentUpgrades.remove(curUO);
if(currentUpgrades.isEmpty()) { // all upgrades are done
@ -110,7 +110,7 @@ class UpgradeManagerNamenode extends UpgradeManager {
public synchronized void completeUpgrade() throws IOException {
// set and write new upgrade state into disk
setUpgradeState(false, FSConstants.LAYOUT_VERSION);
setUpgradeState(false, HdfsConstants.LAYOUT_VERSION);
namesystem.getFSImage().getStorage().writeAll();
currentUpgrades = null;
broadcastCommand = null;

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.UpgradeObject;
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
@ -44,8 +44,8 @@ public abstract class UpgradeObjectNamenode extends UpgradeObject {
public abstract UpgradeCommand processUpgradeCommand(UpgradeCommand command
) throws IOException;
public HdfsConstants.NodeType getType() {
return HdfsConstants.NodeType.NAME_NODE;
public HdfsServerConstants.NodeType getType() {
return HdfsServerConstants.NodeType.NAME_NODE;
}
/**

View File

@ -18,7 +18,7 @@
package org.apache.hadoop.hdfs.server.namenode.metrics;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.metrics2.MetricsSystem;
import org.apache.hadoop.metrics2.annotation.Metric;

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
/**
* Information sent by a subordinate name-node to the active name-node

View File

@ -24,7 +24,7 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.namenode.NNStorage;
@ -53,7 +53,7 @@ public class NamespaceInfo extends StorageInfo {
public NamespaceInfo(int nsID, String clusterID, String bpID,
long cT, int duVersion) {
super(FSConstants.LAYOUT_VERSION, nsID, clusterID, cT);
super(HdfsConstants.LAYOUT_VERSION, nsID, clusterID, cT);
blockPoolID = bpID;
buildVersion = Storage.getBuildVersion();
this.distributedUpgradeVersion = duVersion;

View File

@ -22,15 +22,15 @@ import java.io.DataOutput;
import java.io.IOException;
import java.util.Comparator;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.io.Writable;
import com.google.common.base.Function;
import com.google.common.collect.ComparisonChain;
public class RemoteEditLog implements Writable, Comparable<RemoteEditLog> {
private long startTxId = FSConstants.INVALID_TXID;
private long endTxId = FSConstants.INVALID_TXID;
private long startTxId = HdfsConstants.INVALID_TXID;
private long endTxId = HdfsConstants.INVALID_TXID;
public RemoteEditLog() {
}

View File

@ -25,7 +25,7 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;

View File

@ -40,9 +40,9 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.ipc.RPC;
@ -115,7 +115,7 @@ public class DFSAdmin extends FsShell {
@Override
public void run(Path path) throws IOException {
dfs.setQuota(path, FSConstants.QUOTA_RESET, FSConstants.QUOTA_DONT_SET);
dfs.setQuota(path, HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_DONT_SET);
}
}
@ -161,7 +161,7 @@ public class DFSAdmin extends FsShell {
@Override
public void run(Path path) throws IOException {
dfs.setQuota(path, quota, FSConstants.QUOTA_DONT_SET);
dfs.setQuota(path, quota, HdfsConstants.QUOTA_DONT_SET);
}
}
@ -200,7 +200,7 @@ public class DFSAdmin extends FsShell {
@Override
public void run(Path path) throws IOException {
dfs.setQuota(path, FSConstants.QUOTA_DONT_SET, FSConstants.QUOTA_RESET);
dfs.setQuota(path, HdfsConstants.QUOTA_DONT_SET, HdfsConstants.QUOTA_RESET);
}
}
@ -250,7 +250,7 @@ public class DFSAdmin extends FsShell {
@Override
public void run(Path path) throws IOException {
dfs.setQuota(path, FSConstants.QUOTA_DONT_SET, quota);
dfs.setQuota(path, HdfsConstants.QUOTA_DONT_SET, quota);
}
}
@ -288,7 +288,7 @@ public class DFSAdmin extends FsShell {
long used = ds.getUsed();
long remaining = ds.getRemaining();
long presentCapacity = used + remaining;
boolean mode = dfs.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_GET);
boolean mode = dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_GET);
UpgradeStatusReport status =
dfs.distributedUpgradeProgress(UpgradeAction.GET_STATUS);
@ -361,17 +361,17 @@ public class DFSAdmin extends FsShell {
printUsage("-safemode");
return;
}
FSConstants.SafeModeAction action;
HdfsConstants.SafeModeAction action;
Boolean waitExitSafe = false;
if ("leave".equalsIgnoreCase(argv[idx])) {
action = FSConstants.SafeModeAction.SAFEMODE_LEAVE;
action = HdfsConstants.SafeModeAction.SAFEMODE_LEAVE;
} else if ("enter".equalsIgnoreCase(argv[idx])) {
action = FSConstants.SafeModeAction.SAFEMODE_ENTER;
action = HdfsConstants.SafeModeAction.SAFEMODE_ENTER;
} else if ("get".equalsIgnoreCase(argv[idx])) {
action = FSConstants.SafeModeAction.SAFEMODE_GET;
action = HdfsConstants.SafeModeAction.SAFEMODE_GET;
} else if ("wait".equalsIgnoreCase(argv[idx])) {
action = FSConstants.SafeModeAction.SAFEMODE_GET;
action = HdfsConstants.SafeModeAction.SAFEMODE_GET;
waitExitSafe = true;
} else {
printUsage("-safemode");

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import static org.apache.hadoop.fs.FileContextTestHelper.*;
import org.apache.hadoop.ipc.RemoteException;
import static org.junit.Assert.*;
@ -212,7 +212,7 @@ public class TestFcHdfsSymlink extends FileContextSymlinkBaseTest {
public void testCreateLinkMaxPathLink() throws IOException {
Path dir = new Path(testBaseDir1());
Path file = new Path(testBaseDir1(), "file");
final int maxPathLen = FSConstants.MAX_PATH_LENGTH;
final int maxPathLen = HdfsConstants.MAX_PATH_LENGTH;
final int dirLen = dir.toString().length() + 1;
int len = maxPathLen - dirLen;

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.security.UserGroupInformation;
import org.junit.After;
import org.junit.AfterClass;
@ -108,11 +108,11 @@ public class TestHDFSFileContextMainOperations extends
Path dst2 = getTestRootPath(fc, "test/testOldRenameWithQuota/dstdir/dst2");
createFile(src1);
createFile(src2);
fs.setQuota(src1.getParent(), FSConstants.QUOTA_DONT_SET,
FSConstants.QUOTA_DONT_SET);
fs.setQuota(src1.getParent(), HdfsConstants.QUOTA_DONT_SET,
HdfsConstants.QUOTA_DONT_SET);
fc.mkdir(dst1.getParent(), FileContext.DEFAULT_PERM, true);
fs.setQuota(dst1.getParent(), 2, FSConstants.QUOTA_DONT_SET);
fs.setQuota(dst1.getParent(), 2, HdfsConstants.QUOTA_DONT_SET);
/*
* Test1: src does not exceed quota and dst has no quota check and hence
* accommodates rename
@ -130,7 +130,7 @@ public class TestHDFSFileContextMainOperations extends
* Test3: src exceeds quota and dst has *no* quota to accommodate rename
*/
// src1 has no quota to accommodate new rename node
fs.setQuota(src1.getParent(), 1, FSConstants.QUOTA_DONT_SET);
fs.setQuota(src1.getParent(), 1, HdfsConstants.QUOTA_DONT_SET);
oldRename(dst1, src1, false, true);
}
@ -143,11 +143,11 @@ public class TestHDFSFileContextMainOperations extends
Path dst2 = getTestRootPath(fc, "test/testRenameWithQuota/dstdir/dst2");
createFile(src1);
createFile(src2);
fs.setQuota(src1.getParent(), FSConstants.QUOTA_DONT_SET,
FSConstants.QUOTA_DONT_SET);
fs.setQuota(src1.getParent(), HdfsConstants.QUOTA_DONT_SET,
HdfsConstants.QUOTA_DONT_SET);
fc.mkdir(dst1.getParent(), FileContext.DEFAULT_PERM, true);
fs.setQuota(dst1.getParent(), 2, FSConstants.QUOTA_DONT_SET);
fs.setQuota(dst1.getParent(), 2, HdfsConstants.QUOTA_DONT_SET);
/*
* Test1: src does not exceed quota and dst has no quota check and hence
* accommodates rename
@ -170,7 +170,7 @@ public class TestHDFSFileContextMainOperations extends
* rename to a destination that does not exist
*/
// src1 has no quota to accommodate new rename node
fs.setQuota(src1.getParent(), 1, FSConstants.QUOTA_DONT_SET);
fs.setQuota(src1.getParent(), 1, HdfsConstants.QUOTA_DONT_SET);
rename(dst1, src1, false, false, true, Rename.NONE);
/*
@ -179,9 +179,9 @@ public class TestHDFSFileContextMainOperations extends
* is same as quota needed by src.
*/
// src1 has no quota to accommodate new rename node
fs.setQuota(src1.getParent(), 100, FSConstants.QUOTA_DONT_SET);
fs.setQuota(src1.getParent(), 100, HdfsConstants.QUOTA_DONT_SET);
createFile(src1);
fs.setQuota(src1.getParent(), 1, FSConstants.QUOTA_DONT_SET);
fs.setQuota(src1.getParent(), 1, HdfsConstants.QUOTA_DONT_SET);
rename(dst1, src1, true, true, false, Rename.OVERWRITE);
}
@ -208,7 +208,7 @@ public class TestHDFSFileContextMainOperations extends
createFile(dst1);
// Set quota so that dst1 parent cannot allow under it new files/directories
fs.setQuota(dst1.getParent(), 2, FSConstants.QUOTA_DONT_SET);
fs.setQuota(dst1.getParent(), 2, HdfsConstants.QUOTA_DONT_SET);
// Free up quota for a subsequent rename
fs.delete(dst1, true);
oldRename(src1, dst1, true, false);
@ -237,7 +237,7 @@ public class TestHDFSFileContextMainOperations extends
createFile(dst1);
// Set quota so that dst1 parent cannot allow under it new files/directories
fs.setQuota(dst1.getParent(), 2, FSConstants.QUOTA_DONT_SET);
fs.setQuota(dst1.getParent(), 2, HdfsConstants.QUOTA_DONT_SET);
// Free up quota for a subsequent rename
fs.delete(dst1, true);
rename(src1, dst1, true, true, false, Rename.OVERWRITE);

View File

@ -28,7 +28,7 @@ import java.io.IOException;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FileSystem;
@ -140,8 +140,8 @@ public class BlockReaderTestUtil {
DatanodeInfo[] nodes = testBlock.getLocations();
targetAddr = NetUtils.createSocketAddr(nodes[0].getName());
sock = new Socket();
sock.connect(targetAddr, HdfsConstants.READ_TIMEOUT);
sock.setSoTimeout(HdfsConstants.READ_TIMEOUT);
sock.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT);
sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
return BlockReaderFactory.newBlockReader(
sock, targetAddr.toString()+ ":" + block.getBlockId(), block,

View File

@ -54,7 +54,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
@ -63,7 +63,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@ -670,7 +670,7 @@ public class DFSTestUtil {
final long writeTimeout = dfsClient.getDatanodeWriteTimeout(datanodes.length);
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
NetUtils.getOutputStream(s, writeTimeout),
FSConstants.SMALL_BUFFER_SIZE));
HdfsConstants.SMALL_BUFFER_SIZE));
final DataInputStream in = new DataInputStream(NetUtils.getInputStream(s));
// send the request

View File

@ -27,7 +27,7 @@ import java.util.Random;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.hadoop.hdfs.server.namenode.CreateEditsLog;
import org.apache.hadoop.net.DNS;

View File

@ -47,8 +47,8 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;

View File

@ -23,7 +23,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.*;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
@ -85,7 +85,7 @@ public class TestAbandonBlock {
public void testQuotaUpdatedWhenBlockAbandoned() throws IOException {
DistributedFileSystem dfs = (DistributedFileSystem)fs;
// Setting diskspace quota to 3MB
dfs.setQuota(new Path("/"), FSConstants.QUOTA_DONT_SET, 3 * 1024 * 1024);
dfs.setQuota(new Path("/"), HdfsConstants.QUOTA_DONT_SET, 3 * 1024 * 1024);
// Start writing a file with 2 replicas to ensure each datanode has one.
// Block Size is 1MB.

View File

@ -32,7 +32,7 @@ import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;

View File

@ -25,8 +25,8 @@ import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_NODE;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import com.google.common.collect.Lists;

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.hdfs;
import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.NAME_NODE;
import java.io.File;
import java.io.IOException;
@ -32,8 +32,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import org.apache.hadoop.util.StringUtils;

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.hdfs;
import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.NAME_NODE;
import java.io.File;
@ -27,11 +27,11 @@ import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
/**
* This test ensures the appropriate response (successful or failure) from
@ -198,7 +198,7 @@ public class TestDFSStartupVersions extends TestCase {
return false;
}
// check #3
int softwareLV = FSConstants.LAYOUT_VERSION; // will also be Namenode's LV
int softwareLV = HdfsConstants.LAYOUT_VERSION; // will also be Namenode's LV
int storedLV = datanodeVer.getLayoutVersion();
if (softwareLV == storedLV &&
datanodeVer.getCTime() == namenodeVer.getCTime())

View File

@ -24,11 +24,11 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.NAME_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE;
/**
* This test ensures the appropriate response (successful or failure) from

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.hdfs;
import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.NAME_NODE;
import java.io.File;
import java.io.IOException;
@ -29,7 +29,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.TestParallelImageWrite;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;

View File

@ -30,8 +30,8 @@ import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.StringUtils;
@ -290,7 +290,7 @@ public class TestDFSUpgradeFromImage extends TestCase {
DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
DFSClient dfsClient = dfs.dfs;
//Safemode will be off only after upgrade is complete. Wait for it.
while ( dfsClient.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_GET) ) {
while ( dfsClient.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_GET) ) {
LOG.info("Waiting for SafeMode to be OFF.");
try {
Thread.sleep(1000);

View File

@ -41,7 +41,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
@ -52,7 +52,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto.Builder;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils;
@ -97,8 +97,8 @@ public class TestDataTransferProtocol extends TestCase {
StringUtils.byteToHexString(sendBuf.toByteArray()));
sock = new Socket();
sock.connect(dnAddr, HdfsConstants.READ_TIMEOUT);
sock.setSoTimeout(HdfsConstants.READ_TIMEOUT);
sock.connect(dnAddr, HdfsServerConstants.READ_TIMEOUT);
sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
OutputStream out = sock.getOutputStream();
// Should we excuse

View File

@ -32,7 +32,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;

View File

@ -28,7 +28,7 @@ import java.net.URISyntaxException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.junit.AfterClass;
import org.junit.BeforeClass;

View File

@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs;
import java.net.InetSocketAddress;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.DFSClient;
import junit.framework.TestCase;

View File

@ -24,7 +24,7 @@ import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import static org.apache.hadoop.test.MetricsAsserts.*;

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode;

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.fs.FileSystem;
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.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@ -111,7 +111,7 @@ public class TestFileAppend4 {
// set the soft limit to be 1 second so that the
// namenode triggers lease recovery upon append request
cluster.setLeasePeriod(1000, FSConstants.LEASE_HARDLIMIT_PERIOD);
cluster.setLeasePeriod(1000, HdfsConstants.LEASE_HARDLIMIT_PERIOD);
// Trying recovery
int tries = 60;

View File

@ -40,7 +40,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
@ -301,7 +301,7 @@ public class TestFileCreation extends junit.framework.TestCase {
// wait for the datanode to be declared dead
while (true) {
DatanodeInfo[] info = client.datanodeReport(
FSConstants.DatanodeReportType.LIVE);
HdfsConstants.DatanodeReportType.LIVE);
if (info.length == 0) {
break;
}

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
@ -91,7 +91,7 @@ public class TestFileStatus {
int fileSize, int blockSize) throws IOException {
// Create and write a file that contains three blocks of data
FSDataOutputStream stm = fileSys.create(name, true,
FSConstants.IO_FILE_BUFFER_SIZE, (short)repl, (long)blockSize);
HdfsConstants.IO_FILE_BUFFER_SIZE, (short)repl, (long)blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);

View File

@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.namenode.BackupNode;
import org.apache.hadoop.hdfs.server.namenode.NameNode;

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.TestInterDatanodeProtocol;
@ -129,14 +129,14 @@ public class TestLeaseRecovery extends junit.framework.TestCase {
filestr = "/foo.safemode";
filepath = new Path(filestr);
dfs.create(filepath, (short)1);
cluster.getNameNode().setSafeMode(FSConstants.SafeModeAction.SAFEMODE_ENTER);
cluster.getNameNode().setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
assertTrue(dfs.dfs.exists(filestr));
DFSTestUtil.waitReplication(dfs, filepath, (short)1);
waitLeaseRecovery(cluster);
// verify that we still cannot recover the lease
LeaseManager lm = NameNodeAdapter.getLeaseManager(cluster.getNamesystem());
assertTrue("Found " + lm.countLease() + " lease, expected 1", lm.countLease() == 1);
cluster.getNameNode().setSafeMode(FSConstants.SafeModeAction.SAFEMODE_LEAVE);
cluster.getNameNode().setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
}
finally {
if (cluster != null) {cluster.shutdown();}

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