|
@@ -25,15 +25,17 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAUL
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_DEFAULT;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_DEFAULT;
|
|
@@ -49,15 +51,13 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DAT
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERSIST_BLOCKS_KEY;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERSIST_BLOCKS_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERSIST_BLOCKS_DEFAULT;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERSIST_BLOCKS_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT;
|
|
@@ -150,9 +150,9 @@ import org.apache.hadoop.hdfs.server.common.GenerationStamp;
|
|
|
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.Storage.StorageDirType;
|
|
|
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
|
|
-import org.apache.hadoop.hdfs.server.common.Storage;
|
|
|
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
|
|
|
import org.apache.hadoop.hdfs.server.common.Util;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
|
|
@@ -260,30 +260,28 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
static final int DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED = 100;
|
|
|
static int BLOCK_DELETION_INCREMENT = 1000;
|
|
|
- private boolean isPermissionEnabled;
|
|
|
- private boolean persistBlocks;
|
|
|
- private UserGroupInformation fsOwner;
|
|
|
- private String supergroup;
|
|
|
- private boolean standbyShouldCheckpoint;
|
|
|
+ private final boolean isPermissionEnabled;
|
|
|
+ private final boolean persistBlocks;
|
|
|
+ private final UserGroupInformation fsOwner;
|
|
|
+ private final String supergroup;
|
|
|
+ private final boolean standbyShouldCheckpoint;
|
|
|
|
|
|
// Scan interval is not configurable.
|
|
|
private static final long DELEGATION_TOKEN_REMOVER_SCAN_INTERVAL =
|
|
|
TimeUnit.MILLISECONDS.convert(1, TimeUnit.HOURS);
|
|
|
- private DelegationTokenSecretManager dtSecretManager;
|
|
|
- private boolean alwaysUseDelegationTokensForTests;
|
|
|
+ private final DelegationTokenSecretManager dtSecretManager;
|
|
|
+ private final boolean alwaysUseDelegationTokensForTests;
|
|
|
|
|
|
|
|
|
- //
|
|
|
- // Stores the correct file name hierarchy
|
|
|
- //
|
|
|
+ /** The namespace tree. */
|
|
|
FSDirectory dir;
|
|
|
- private BlockManager blockManager;
|
|
|
- private DatanodeStatistics datanodeStatistics;
|
|
|
+ private final BlockManager blockManager;
|
|
|
+ private final DatanodeStatistics datanodeStatistics;
|
|
|
|
|
|
// Block pool ID used by this namenode
|
|
|
private String blockPoolId;
|
|
|
|
|
|
- LeaseManager leaseManager = new LeaseManager(this);
|
|
|
+ final LeaseManager leaseManager = new LeaseManager(this);
|
|
|
|
|
|
Daemon smmthread = null; // SafeModeMonitor thread
|
|
|
|
|
@@ -291,23 +289,23 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
private volatile boolean hasResourcesAvailable = false;
|
|
|
private volatile boolean fsRunning = true;
|
|
|
- long systemStart = 0;
|
|
|
+
|
|
|
+ /** The start time of the namesystem. */
|
|
|
+ private final long startTime = now();
|
|
|
|
|
|
- //resourceRecheckInterval is how often namenode checks for the disk space availability
|
|
|
- private long resourceRecheckInterval;
|
|
|
+ /** The interval of namenode checking for the disk space availability */
|
|
|
+ private final long resourceRecheckInterval;
|
|
|
|
|
|
// The actual resource checker instance.
|
|
|
NameNodeResourceChecker nnResourceChecker;
|
|
|
|
|
|
- private FsServerDefaults serverDefaults;
|
|
|
-
|
|
|
- private boolean supportAppends;
|
|
|
- private ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure =
|
|
|
- ReplaceDatanodeOnFailure.DEFAULT;
|
|
|
+ private final FsServerDefaults serverDefaults;
|
|
|
+ private final boolean supportAppends;
|
|
|
+ private final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
|
|
|
|
|
|
private volatile SafeModeInfo safeMode; // safe mode information
|
|
|
|
|
|
- private long maxFsObjects = 0; // maximum number of fs objects
|
|
|
+ private final long maxFsObjects; // maximum number of fs objects
|
|
|
|
|
|
/**
|
|
|
* The global generation stamp for this file system.
|
|
@@ -315,10 +313,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
private final GenerationStamp generationStamp = new GenerationStamp();
|
|
|
|
|
|
// precision of access times.
|
|
|
- private long accessTimePrecision = 0;
|
|
|
+ private final long accessTimePrecision;
|
|
|
|
|
|
- // lock to protect FSNamesystem.
|
|
|
- private ReentrantReadWriteLock fsLock;
|
|
|
+ /** Lock to protect FSNamesystem. */
|
|
|
+ private ReentrantReadWriteLock fsLock = new ReentrantReadWriteLock(true);
|
|
|
|
|
|
/**
|
|
|
* Used when this NN is in standby state to read from the shared edit log.
|
|
@@ -336,9 +334,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
*/
|
|
|
private HAContext haContext;
|
|
|
|
|
|
- private boolean haEnabled;
|
|
|
-
|
|
|
- private final Configuration conf;
|
|
|
+ private final boolean haEnabled;
|
|
|
|
|
|
/**
|
|
|
* Instantiates an FSNamesystem loaded from the image and edits
|
|
@@ -390,9 +386,71 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
* @throws IOException on bad configuration
|
|
|
*/
|
|
|
FSNamesystem(Configuration conf, FSImage fsImage) throws IOException {
|
|
|
- this.conf = conf;
|
|
|
try {
|
|
|
- initialize(conf, fsImage);
|
|
|
+ resourceRecheckInterval = conf.getLong(
|
|
|
+ DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY,
|
|
|
+ DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_DEFAULT);
|
|
|
+
|
|
|
+ this.blockManager = new BlockManager(this, this, conf);
|
|
|
+ this.datanodeStatistics = blockManager.getDatanodeManager().getDatanodeStatistics();
|
|
|
+
|
|
|
+ this.fsOwner = UserGroupInformation.getCurrentUser();
|
|
|
+ this.supergroup = conf.get(DFS_PERMISSIONS_SUPERUSERGROUP_KEY,
|
|
|
+ DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT);
|
|
|
+ this.isPermissionEnabled = conf.getBoolean(DFS_PERMISSIONS_ENABLED_KEY,
|
|
|
+ DFS_PERMISSIONS_ENABLED_DEFAULT);
|
|
|
+ LOG.info("fsOwner = " + fsOwner);
|
|
|
+ LOG.info("supergroup = " + supergroup);
|
|
|
+ LOG.info("isPermissionEnabled = " + isPermissionEnabled);
|
|
|
+
|
|
|
+ final boolean persistBlocks = conf.getBoolean(DFS_PERSIST_BLOCKS_KEY,
|
|
|
+ DFS_PERSIST_BLOCKS_DEFAULT);
|
|
|
+ // block allocation has to be persisted in HA using a shared edits directory
|
|
|
+ // so that the standby has up-to-date namespace information
|
|
|
+ String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
|
|
|
+ this.haEnabled = HAUtil.isHAEnabled(conf, nameserviceId);
|
|
|
+ this.persistBlocks = persistBlocks || (haEnabled && HAUtil.usesSharedEditsDir(conf));
|
|
|
+
|
|
|
+ // Sanity check the HA-related config.
|
|
|
+ if (nameserviceId != null) {
|
|
|
+ LOG.info("Determined nameservice ID: " + nameserviceId);
|
|
|
+ }
|
|
|
+ LOG.info("HA Enabled: " + haEnabled);
|
|
|
+ if (!haEnabled && HAUtil.usesSharedEditsDir(conf)) {
|
|
|
+ LOG.warn("Configured NNs:\n" + DFSUtil.nnAddressesAsString(conf));
|
|
|
+ throw new IOException("Invalid configuration: a shared edits dir " +
|
|
|
+ "must not be specified if HA is not enabled.");
|
|
|
+ }
|
|
|
+
|
|
|
+ this.serverDefaults = new FsServerDefaults(
|
|
|
+ conf.getLongBytes(DFS_BLOCK_SIZE_KEY, DFS_BLOCK_SIZE_DEFAULT),
|
|
|
+ conf.getInt(DFS_BYTES_PER_CHECKSUM_KEY, DFS_BYTES_PER_CHECKSUM_DEFAULT),
|
|
|
+ conf.getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY, DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT),
|
|
|
+ (short) conf.getInt(DFS_REPLICATION_KEY, DFS_REPLICATION_DEFAULT),
|
|
|
+ conf.getInt(IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT));
|
|
|
+
|
|
|
+ this.maxFsObjects = conf.getLong(DFS_NAMENODE_MAX_OBJECTS_KEY,
|
|
|
+ DFS_NAMENODE_MAX_OBJECTS_DEFAULT);
|
|
|
+
|
|
|
+ this.accessTimePrecision = conf.getLong(DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 0);
|
|
|
+ this.supportAppends = conf.getBoolean(DFS_SUPPORT_APPEND_KEY, DFS_SUPPORT_APPEND_DEFAULT);
|
|
|
+ LOG.info("Append Enabled: " + haEnabled);
|
|
|
+
|
|
|
+ this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
|
|
|
+
|
|
|
+ this.standbyShouldCheckpoint = conf.getBoolean(
|
|
|
+ DFS_HA_STANDBY_CHECKPOINTS_KEY, DFS_HA_STANDBY_CHECKPOINTS_DEFAULT);
|
|
|
+
|
|
|
+ // For testing purposes, allow the DT secret manager to be started regardless
|
|
|
+ // of whether security is enabled.
|
|
|
+ alwaysUseDelegationTokensForTests = conf.getBoolean(
|
|
|
+ DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY,
|
|
|
+ DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT);
|
|
|
+
|
|
|
+ this.dtSecretManager = createDelegationTokenSecretManager(conf);
|
|
|
+ this.dir = new FSDirectory(fsImage, this, conf);
|
|
|
+ this.safeMode = new SafeModeInfo(conf);
|
|
|
+
|
|
|
} catch(IOException e) {
|
|
|
LOG.error(getClass().getSimpleName() + " initialization failed.", e);
|
|
|
close();
|
|
@@ -400,24 +458,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Initialize FSNamesystem.
|
|
|
- */
|
|
|
- private void initialize(Configuration conf, FSImage fsImage)
|
|
|
- throws IOException {
|
|
|
- resourceRecheckInterval = conf.getLong(
|
|
|
- DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY,
|
|
|
- DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_DEFAULT);
|
|
|
- this.systemStart = now();
|
|
|
- this.blockManager = new BlockManager(this, this, conf);
|
|
|
- this.datanodeStatistics = blockManager.getDatanodeManager().getDatanodeStatistics();
|
|
|
- this.fsLock = new ReentrantReadWriteLock(true); // fair locking
|
|
|
- setConfigurationParameters(conf);
|
|
|
- dtSecretManager = createDelegationTokenSecretManager(conf);
|
|
|
- this.dir = new FSDirectory(fsImage, this, conf);
|
|
|
- this.safeMode = new SafeModeInfo(conf);
|
|
|
- }
|
|
|
-
|
|
|
void loadFSImage(StartupOption startOpt, FSImage fsImage, boolean haEnabled)
|
|
|
throws IOException {
|
|
|
// format before starting up if requested
|
|
@@ -601,13 +641,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
/** Start services required in standby state */
|
|
|
- void startStandbyServices() {
|
|
|
+ void startStandbyServices(final Configuration conf) {
|
|
|
LOG.info("Starting services required for standby state");
|
|
|
if (!dir.fsImage.editLog.isOpenForRead()) {
|
|
|
// During startup, we're already open for read.
|
|
|
dir.fsImage.editLog.initSharedJournalsForRead();
|
|
|
}
|
|
|
- editLogTailer = new EditLogTailer(this);
|
|
|
+ editLogTailer = new EditLogTailer(this, conf);
|
|
|
editLogTailer.start();
|
|
|
if (standbyShouldCheckpoint) {
|
|
|
standbyCheckpointer = new StandbyCheckpointer(conf, this);
|
|
@@ -768,10 +808,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
DFS_NAMENODE_SHARED_EDITS_DIR_KEY);
|
|
|
return Util.stringCollectionAsURIs(dirNames);
|
|
|
}
|
|
|
-
|
|
|
- public Configuration getConf() {
|
|
|
- return conf;
|
|
|
- }
|
|
|
|
|
|
@Override
|
|
|
public void readLock() {
|
|
@@ -806,69 +842,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
return hasReadLock() || hasWriteLock();
|
|
|
}
|
|
|
|
|
|
-
|
|
|
- /**
|
|
|
- * Initializes some of the members from configuration
|
|
|
- */
|
|
|
- private void setConfigurationParameters(Configuration conf)
|
|
|
- throws IOException {
|
|
|
- fsOwner = UserGroupInformation.getCurrentUser();
|
|
|
-
|
|
|
- LOG.info("fsOwner=" + fsOwner);
|
|
|
-
|
|
|
- this.supergroup = conf.get(DFS_PERMISSIONS_SUPERUSERGROUP_KEY,
|
|
|
- DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT);
|
|
|
- this.isPermissionEnabled = conf.getBoolean(DFS_PERMISSIONS_ENABLED_KEY,
|
|
|
- DFS_PERMISSIONS_ENABLED_DEFAULT);
|
|
|
- LOG.info("supergroup=" + supergroup);
|
|
|
- LOG.info("isPermissionEnabled=" + isPermissionEnabled);
|
|
|
-
|
|
|
- this.persistBlocks = conf.getBoolean(DFS_PERSIST_BLOCKS_KEY,
|
|
|
- DFS_PERSIST_BLOCKS_DEFAULT);
|
|
|
- // block allocation has to be persisted in HA using a shared edits directory
|
|
|
- // so that the standby has up-to-date namespace information
|
|
|
- String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
|
|
|
- this.haEnabled = HAUtil.isHAEnabled(conf, nameserviceId);
|
|
|
- this.persistBlocks |= haEnabled && HAUtil.usesSharedEditsDir(conf);
|
|
|
-
|
|
|
- // Sanity check the HA-related config.
|
|
|
- if (nameserviceId != null) {
|
|
|
- LOG.info("Determined nameservice ID: " + nameserviceId);
|
|
|
- }
|
|
|
- LOG.info("HA Enabled: " + haEnabled);
|
|
|
- if (!haEnabled && HAUtil.usesSharedEditsDir(conf)) {
|
|
|
- LOG.warn("Configured NNs:\n" + DFSUtil.nnAddressesAsString(conf));
|
|
|
- throw new IOException("Invalid configuration: a shared edits dir " +
|
|
|
- "must not be specified if HA is not enabled.");
|
|
|
- }
|
|
|
-
|
|
|
- this.serverDefaults = new FsServerDefaults(
|
|
|
- conf.getLongBytes(DFS_BLOCK_SIZE_KEY, DFS_BLOCK_SIZE_DEFAULT),
|
|
|
- conf.getInt(DFS_BYTES_PER_CHECKSUM_KEY, DFS_BYTES_PER_CHECKSUM_DEFAULT),
|
|
|
- conf.getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY, DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT),
|
|
|
- (short) conf.getInt(DFS_REPLICATION_KEY, DFS_REPLICATION_DEFAULT),
|
|
|
- conf.getInt(IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT));
|
|
|
-
|
|
|
- this.maxFsObjects = conf.getLong(DFS_NAMENODE_MAX_OBJECTS_KEY,
|
|
|
- DFS_NAMENODE_MAX_OBJECTS_DEFAULT);
|
|
|
-
|
|
|
- this.accessTimePrecision = conf.getLong(DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 0);
|
|
|
- this.supportAppends = conf.getBoolean(DFS_SUPPORT_APPEND_KEY,
|
|
|
- DFS_SUPPORT_APPEND_DEFAULT);
|
|
|
-
|
|
|
- this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
|
|
|
-
|
|
|
- this.standbyShouldCheckpoint = conf.getBoolean(
|
|
|
- DFS_HA_STANDBY_CHECKPOINTS_KEY,
|
|
|
- DFS_HA_STANDBY_CHECKPOINTS_DEFAULT);
|
|
|
-
|
|
|
- // For testing purposes, allow the DT secret manager to be started regardless
|
|
|
- // of whether security is enabled.
|
|
|
- alwaysUseDelegationTokensForTests =
|
|
|
- conf.getBoolean(DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY,
|
|
|
- DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT);
|
|
|
- }
|
|
|
-
|
|
|
NamespaceInfo getNamespaceInfo() {
|
|
|
readLock();
|
|
|
try {
|
|
@@ -2761,7 +2734,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
private Lease reassignLease(Lease lease, String src, String newHolder,
|
|
|
- INodeFileUnderConstruction pendingFile) throws IOException {
|
|
|
+ INodeFileUnderConstruction pendingFile) {
|
|
|
assert hasWriteLock();
|
|
|
if(newHolder == null)
|
|
|
return lease;
|
|
@@ -3329,7 +3302,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
Date getStartTime() {
|
|
|
- return new Date(systemStart);
|
|
|
+ return new Date(startTime);
|
|
|
}
|
|
|
|
|
|
void finalizeUpgrade() throws IOException {
|
|
@@ -3506,7 +3479,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
if (!isPopulatingReplQueues() && !isInStandbyState()) {
|
|
|
initializeReplQueues();
|
|
|
}
|
|
|
- long timeInSafemode = now() - systemStart;
|
|
|
+ long timeInSafemode = now() - startTime;
|
|
|
NameNode.stateChangeLog.info("STATE* Leaving safe mode after "
|
|
|
+ timeInSafemode/1000 + " secs.");
|
|
|
NameNode.getNameNodeMetrics().setSafeModeTime((int) timeInSafemode);
|
|
@@ -4876,7 +4849,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
*
|
|
|
* @param key new delegation key.
|
|
|
*/
|
|
|
- public void logUpdateMasterKey(DelegationKey key) throws IOException {
|
|
|
+ public void logUpdateMasterKey(DelegationKey key) {
|
|
|
|
|
|
assert !isInSafeMode() :
|
|
|
"this should never be called while in safemode, since we stop " +
|
|
@@ -4889,7 +4862,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
private void logReassignLease(String leaseHolder, String src,
|
|
|
- String newHolder) throws IOException {
|
|
|
+ String newHolder) {
|
|
|
writeLock();
|
|
|
try {
|
|
|
getEditLog().logReassignLease(leaseHolder, src, newHolder);
|