001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hdfs.server.namenode; 019 020import static org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion; 021import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT; 022import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY; 023import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT; 024import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; 025import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT; 026import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY; 027import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT; 028import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY; 029import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT; 030import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY; 031import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT; 032import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY; 033import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_DEFAULT; 034import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY; 035import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_DEFAULT; 036import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_KEY; 037import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT; 038import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY; 039import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY; 040import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT; 041import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY; 042import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_DEFAULT; 043import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_KEY; 044import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT; 045import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY; 046import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME; 047import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT; 048import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_KEY; 049import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT; 050import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY; 051import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT; 052import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY; 053import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT; 054import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY; 055import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY; 056import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY; 057import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROLL_CHECK_INTERVAL_MS; 058import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROLL_CHECK_INTERVAL_MS_DEFAULT; 059import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROLL_MULTIPLIER_THRESHOLD; 060import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROLL_MULTIPLIER_THRESHOLD_DEFAULT; 061import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_DEFAULT; 062import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_KEY; 063import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC; 064import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC_DEFAULT; 065import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY; 066import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_DEFAULT; 067import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY; 068import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY; 069import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT; 070import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY; 071import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REPL_QUEUE_THRESHOLD_PCT_KEY; 072import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_DEFAULT; 073import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY; 074import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_EXPIRYTIME_MILLIS_DEFAULT; 075import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_EXPIRYTIME_MILLIS_KEY; 076import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_DEFAULT; 077import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_KEY; 078import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY; 079import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_DEFAULT; 080import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY; 081import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_DEFAULT; 082import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY; 083import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY; 084import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_DEFAULT; 085import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY; 086import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT; 087import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY; 088import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT; 089import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY; 090import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT; 091import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SUPPORT_APPEND_KEY; 092import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.SECURITY_XATTR_UNREADABLE_BY_SUPERUSER; 093import static org.apache.hadoop.util.Time.now; 094import static org.apache.hadoop.util.Time.monotonicNow; 095 096import java.io.BufferedWriter; 097import java.io.ByteArrayInputStream; 098import java.io.DataInput; 099import java.io.DataInputStream; 100import java.io.DataOutputStream; 101import java.io.File; 102import java.io.FileNotFoundException; 103import java.io.FileOutputStream; 104import java.io.IOException; 105import java.io.OutputStreamWriter; 106import java.io.PrintWriter; 107import java.io.StringWriter; 108import java.lang.management.ManagementFactory; 109import java.net.InetAddress; 110import java.net.URI; 111import java.security.GeneralSecurityException; 112import java.util.ArrayList; 113import java.util.Arrays; 114import java.util.Collection; 115import java.util.Collections; 116import java.util.Date; 117import java.util.EnumSet; 118import java.util.HashMap; 119import java.util.HashSet; 120import java.util.Iterator; 121import java.util.LinkedHashSet; 122import java.util.List; 123import java.util.Map; 124import java.util.Set; 125import java.util.TreeMap; 126import java.util.concurrent.TimeUnit; 127import java.util.concurrent.locks.Condition; 128import java.util.concurrent.locks.ReentrantLock; 129import java.util.concurrent.locks.ReentrantReadWriteLock; 130 131import javax.management.NotCompliantMBeanException; 132import javax.management.ObjectName; 133import javax.management.StandardMBean; 134 135import org.apache.commons.logging.Log; 136import org.apache.commons.logging.LogFactory; 137import org.apache.commons.logging.impl.Log4JLogger; 138import org.apache.hadoop.HadoopIllegalArgumentException; 139import org.apache.hadoop.classification.InterfaceAudience; 140import org.apache.hadoop.conf.Configuration; 141import org.apache.hadoop.crypto.CipherSuite; 142import org.apache.hadoop.crypto.CryptoProtocolVersion; 143import org.apache.hadoop.crypto.key.KeyProvider; 144import org.apache.hadoop.crypto.CryptoCodec; 145import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension; 146import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries; 147import org.apache.hadoop.fs.CacheFlag; 148import org.apache.hadoop.fs.ContentSummary; 149import org.apache.hadoop.fs.CreateFlag; 150import org.apache.hadoop.fs.FileAlreadyExistsException; 151import org.apache.hadoop.fs.FileEncryptionInfo; 152import org.apache.hadoop.fs.FileStatus; 153import org.apache.hadoop.fs.FileSystem; 154import org.apache.hadoop.fs.FsServerDefaults; 155import org.apache.hadoop.fs.InvalidPathException; 156import org.apache.hadoop.fs.Options; 157import org.apache.hadoop.fs.ParentNotDirectoryException; 158import org.apache.hadoop.fs.Path; 159import org.apache.hadoop.fs.UnresolvedLinkException; 160import org.apache.hadoop.fs.XAttr; 161import org.apache.hadoop.fs.XAttrSetFlag; 162import org.apache.hadoop.fs.permission.AclEntry; 163import org.apache.hadoop.fs.permission.AclStatus; 164import org.apache.hadoop.fs.permission.FsAction; 165import org.apache.hadoop.fs.permission.FsPermission; 166import org.apache.hadoop.fs.permission.PermissionStatus; 167import org.apache.hadoop.fs.StorageType; 168import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; 169import org.apache.hadoop.ha.ServiceFailedException; 170import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; 171import org.apache.hadoop.hdfs.DFSConfigKeys; 172import org.apache.hadoop.hdfs.DFSUtil; 173import org.apache.hadoop.hdfs.HAUtil; 174import org.apache.hadoop.hdfs.HdfsConfiguration; 175import org.apache.hadoop.hdfs.UnknownCryptoProtocolVersionException; 176import org.apache.hadoop.hdfs.XAttrHelper; 177import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; 178import org.apache.hadoop.hdfs.protocol.Block; 179import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry; 180import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; 181import org.apache.hadoop.hdfs.protocol.CachePoolEntry; 182import org.apache.hadoop.hdfs.protocol.CachePoolInfo; 183import org.apache.hadoop.hdfs.protocol.ClientProtocol; 184import org.apache.hadoop.hdfs.protocol.DatanodeID; 185import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 186import org.apache.hadoop.hdfs.protocol.DirectoryListing; 187import org.apache.hadoop.hdfs.protocol.EncryptionZone; 188import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 189import org.apache.hadoop.hdfs.protocol.HdfsConstants; 190import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus; 191import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; 192import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; 193import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; 194import org.apache.hadoop.hdfs.protocol.LocatedBlock; 195import org.apache.hadoop.hdfs.protocol.LocatedBlocks; 196import org.apache.hadoop.hdfs.protocol.QuotaExceededException; 197import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException; 198import org.apache.hadoop.hdfs.protocol.RollingUpgradeException; 199import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo; 200import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; 201import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; 202import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; 203import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure; 204import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager; 205import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode; 206import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; 207import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager; 208import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager.SecretManagerState; 209import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; 210import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; 211import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; 212import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; 213import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; 214import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; 215import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; 216import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics; 217import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; 218import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; 219import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; 220import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption; 221import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; 222import org.apache.hadoop.hdfs.server.common.Storage; 223import org.apache.hadoop.hdfs.server.common.Storage.StorageDirType; 224import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; 225import org.apache.hadoop.hdfs.server.common.Util; 226import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection; 227import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; 228import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream; 229import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease; 230import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile; 231import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory; 232import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer; 233import org.apache.hadoop.hdfs.server.namenode.ha.HAContext; 234import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer; 235import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean; 236import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics; 237import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot; 238import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager; 239import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase; 240import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress; 241import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter; 242import org.apache.hadoop.hdfs.server.namenode.startupprogress.Status; 243import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step; 244import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType; 245import org.apache.hadoop.hdfs.server.namenode.top.TopAuditLogger; 246import org.apache.hadoop.hdfs.server.namenode.top.TopConf; 247import org.apache.hadoop.hdfs.server.namenode.top.metrics.TopMetrics; 248import org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager; 249import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods; 250import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand; 251import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; 252import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; 253import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse; 254import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat; 255import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand; 256import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration; 257import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; 258import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; 259import org.apache.hadoop.hdfs.server.protocol.StorageReport; 260import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary; 261import org.apache.hadoop.io.EnumSetWritable; 262import org.apache.hadoop.io.IOUtils; 263import org.apache.hadoop.io.Text; 264import org.apache.hadoop.ipc.RetriableException; 265import org.apache.hadoop.ipc.RetryCache; 266import org.apache.hadoop.ipc.Server; 267import org.apache.hadoop.ipc.StandbyException; 268import org.apache.hadoop.metrics2.annotation.Metric; 269import org.apache.hadoop.metrics2.annotation.Metrics; 270import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; 271import org.apache.hadoop.metrics2.util.MBeans; 272import org.apache.hadoop.net.NetworkTopology; 273import org.apache.hadoop.net.Node; 274import org.apache.hadoop.net.NodeBase; 275import org.apache.hadoop.security.AccessControlException; 276import org.apache.hadoop.security.UserGroupInformation; 277import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; 278import org.apache.hadoop.security.token.SecretManager.InvalidToken; 279import org.apache.hadoop.security.token.Token; 280import org.apache.hadoop.security.token.TokenIdentifier; 281import org.apache.hadoop.security.token.delegation.DelegationKey; 282import org.apache.hadoop.util.ChunkedArrayList; 283import org.apache.hadoop.util.Daemon; 284import org.apache.hadoop.util.DataChecksum; 285import org.apache.hadoop.util.ReflectionUtils; 286import org.apache.hadoop.util.StringUtils; 287import org.apache.hadoop.util.VersionInfo; 288import org.apache.log4j.Appender; 289import org.apache.log4j.AsyncAppender; 290import org.apache.log4j.Logger; 291import org.codehaus.jackson.map.ObjectMapper; 292import org.mortbay.util.ajax.JSON; 293 294import com.google.common.annotations.VisibleForTesting; 295import com.google.common.base.Charsets; 296import com.google.common.base.Preconditions; 297import com.google.common.collect.ImmutableMap; 298import com.google.common.collect.Lists; 299 300/*************************************************** 301 * FSNamesystem does the actual bookkeeping work for the 302 * DataNode. 303 * 304 * It tracks several important tables. 305 * 306 * 1) valid fsname --> blocklist (kept on disk, logged) 307 * 2) Set of all valid blocks (inverted #1) 308 * 3) block --> machinelist (kept in memory, rebuilt dynamically from reports) 309 * 4) machine --> blocklist (inverted #2) 310 * 5) LRU cache of updated-heartbeat machines 311 ***************************************************/ 312@InterfaceAudience.Private 313@Metrics(context="dfs") 314public class FSNamesystem implements Namesystem, FSNamesystemMBean, 315 NameNodeMXBean { 316 public static final Log LOG = LogFactory.getLog(FSNamesystem.class); 317 318 private static final ThreadLocal<StringBuilder> auditBuffer = 319 new ThreadLocal<StringBuilder>() { 320 @Override 321 protected StringBuilder initialValue() { 322 return new StringBuilder(); 323 } 324 }; 325 326 private final BlockIdManager blockIdManager; 327 328 @VisibleForTesting 329 public boolean isAuditEnabled() { 330 return !isDefaultAuditLogger || auditLog.isInfoEnabled(); 331 } 332 333 private void logAuditEvent(boolean succeeded, String cmd, String src) 334 throws IOException { 335 logAuditEvent(succeeded, cmd, src, null, null); 336 } 337 338 private void logAuditEvent(boolean succeeded, String cmd, String src, 339 String dst, HdfsFileStatus stat) throws IOException { 340 if (isAuditEnabled() && isExternalInvocation()) { 341 logAuditEvent(succeeded, getRemoteUser(), getRemoteIp(), 342 cmd, src, dst, stat); 343 } 344 } 345 346 private void logAuditEvent(boolean succeeded, 347 UserGroupInformation ugi, InetAddress addr, String cmd, String src, 348 String dst, HdfsFileStatus stat) { 349 FileStatus status = null; 350 if (stat != null) { 351 Path symlink = stat.isSymlink() ? new Path(stat.getSymlink()) : null; 352 Path path = dst != null ? new Path(dst) : new Path(src); 353 status = new FileStatus(stat.getLen(), stat.isDir(), 354 stat.getReplication(), stat.getBlockSize(), stat.getModificationTime(), 355 stat.getAccessTime(), stat.getPermission(), stat.getOwner(), 356 stat.getGroup(), symlink, path); 357 } 358 for (AuditLogger logger : auditLoggers) { 359 if (logger instanceof HdfsAuditLogger) { 360 HdfsAuditLogger hdfsLogger = (HdfsAuditLogger) logger; 361 hdfsLogger.logAuditEvent(succeeded, ugi.toString(), addr, cmd, src, dst, 362 status, ugi, dtSecretManager); 363 } else { 364 logger.logAuditEvent(succeeded, ugi.toString(), addr, 365 cmd, src, dst, status); 366 } 367 } 368 } 369 370 /** 371 * Logger for audit events, noting successful FSNamesystem operations. Emits 372 * to FSNamesystem.audit at INFO. Each event causes a set of tab-separated 373 * <code>key=value</code> pairs to be written for the following properties: 374 * <code> 375 * ugi=<ugi in RPC> 376 * ip=<remote IP> 377 * cmd=<command> 378 * src=<src path> 379 * dst=<dst path (optional)> 380 * perm=<permissions (optional)> 381 * </code> 382 */ 383 public static final Log auditLog = LogFactory.getLog( 384 FSNamesystem.class.getName() + ".audit"); 385 386 static final int DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED = 100; 387 static int BLOCK_DELETION_INCREMENT = 1000; 388 private final boolean isPermissionEnabled; 389 private final UserGroupInformation fsOwner; 390 private final String supergroup; 391 private final boolean standbyShouldCheckpoint; 392 393 // Scan interval is not configurable. 394 private static final long DELEGATION_TOKEN_REMOVER_SCAN_INTERVAL = 395 TimeUnit.MILLISECONDS.convert(1, TimeUnit.HOURS); 396 final DelegationTokenSecretManager dtSecretManager; 397 private final boolean alwaysUseDelegationTokensForTests; 398 399 private static final Step STEP_AWAITING_REPORTED_BLOCKS = 400 new Step(StepType.AWAITING_REPORTED_BLOCKS); 401 402 // Tracks whether the default audit logger is the only configured audit 403 // logger; this allows isAuditEnabled() to return false in case the 404 // underlying logger is disabled, and avoid some unnecessary work. 405 private final boolean isDefaultAuditLogger; 406 private final List<AuditLogger> auditLoggers; 407 408 /** The namespace tree. */ 409 FSDirectory dir; 410 private final BlockManager blockManager; 411 private final SnapshotManager snapshotManager; 412 private final CacheManager cacheManager; 413 private final DatanodeStatistics datanodeStatistics; 414 415 private String nameserviceId; 416 417 private volatile RollingUpgradeInfo rollingUpgradeInfo = null; 418 /** 419 * A flag that indicates whether the checkpointer should checkpoint a rollback 420 * fsimage. The edit log tailer sets this flag. The checkpoint will create a 421 * rollback fsimage if the flag is true, and then change the flag to false. 422 */ 423 private volatile boolean needRollbackFsImage; 424 425 // Block pool ID used by this namenode 426 private String blockPoolId; 427 428 final LeaseManager leaseManager = new LeaseManager(this); 429 430 volatile Daemon smmthread = null; // SafeModeMonitor thread 431 432 Daemon nnrmthread = null; // NamenodeResourceMonitor thread 433 434 Daemon nnEditLogRoller = null; // NameNodeEditLogRoller thread 435 436 // A daemon to periodically clean up corrupt lazyPersist files 437 // from the name space. 438 Daemon lazyPersistFileScrubber = null; 439 /** 440 * When an active namenode will roll its own edit log, in # edits 441 */ 442 private final long editLogRollerThreshold; 443 /** 444 * Check interval of an active namenode's edit log roller thread 445 */ 446 private final int editLogRollerInterval; 447 448 /** 449 * How frequently we scan and unlink corrupt lazyPersist files. 450 * (In seconds) 451 */ 452 private final int lazyPersistFileScrubIntervalSec; 453 454 private volatile boolean hasResourcesAvailable = false; 455 private volatile boolean fsRunning = true; 456 457 /** The start time of the namesystem. */ 458 private final long startTime = now(); 459 460 /** The interval of namenode checking for the disk space availability */ 461 private final long resourceRecheckInterval; 462 463 // The actual resource checker instance. 464 NameNodeResourceChecker nnResourceChecker; 465 466 private final FsServerDefaults serverDefaults; 467 private final boolean supportAppends; 468 private final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure; 469 470 private volatile SafeModeInfo safeMode; // safe mode information 471 472 private final long maxFsObjects; // maximum number of fs objects 473 474 private final long minBlockSize; // minimum block size 475 private final long maxBlocksPerFile; // maximum # of blocks per file 476 477 // precision of access times. 478 private final long accessTimePrecision; 479 480 /** Lock to protect FSNamesystem. */ 481 private final FSNamesystemLock fsLock; 482 483 /** 484 * Checkpoint lock to protect FSNamesystem modification on standby NNs. 485 * Unlike fsLock, it does not affect block updates. On active NNs, this lock 486 * does not provide proper protection, because there are operations that 487 * modify both block and name system state. Even on standby, fsLock is 488 * used when block state changes need to be blocked. 489 */ 490 private final ReentrantLock cpLock; 491 492 /** 493 * Used when this NN is in standby state to read from the shared edit log. 494 */ 495 private EditLogTailer editLogTailer = null; 496 497 /** 498 * Used when this NN is in standby state to perform checkpoints. 499 */ 500 private StandbyCheckpointer standbyCheckpointer; 501 502 /** 503 * Reference to the NN's HAContext object. This is only set once 504 * {@link #startCommonServices(Configuration, HAContext)} is called. 505 */ 506 private HAContext haContext; 507 508 private final boolean haEnabled; 509 510 /** flag indicating whether replication queues have been initialized */ 511 boolean initializedReplQueues = false; 512 513 /** 514 * Whether the namenode is in the middle of starting the active service 515 */ 516 private volatile boolean startingActiveService = false; 517 518 private final RetryCache retryCache; 519 520 private KeyProviderCryptoExtension provider = null; 521 522 private volatile boolean imageLoaded = false; 523 private final Condition cond; 524 525 private final FSImage fsImage; 526 527 private final TopConf topConf; 528 private TopMetrics topMetrics; 529 530 private INodeAttributeProvider inodeAttributeProvider; 531 532 /** 533 * Notify that loading of this FSDirectory is complete, and 534 * it is imageLoaded for use 535 */ 536 void imageLoadComplete() { 537 Preconditions.checkState(!imageLoaded, "FSDirectory already loaded"); 538 setImageLoaded(); 539 } 540 541 void setImageLoaded() { 542 if(imageLoaded) return; 543 writeLock(); 544 try { 545 setImageLoaded(true); 546 dir.markNameCacheInitialized(); 547 cond.signalAll(); 548 } finally { 549 writeUnlock(); 550 } 551 } 552 553 //This is for testing purposes only 554 @VisibleForTesting 555 boolean isImageLoaded() { 556 return imageLoaded; 557 } 558 559 // exposed for unit tests 560 protected void setImageLoaded(boolean flag) { 561 imageLoaded = flag; 562 } 563 564 /** 565 * Block until the object is imageLoaded to be used. 566 */ 567 void waitForLoadingFSImage() { 568 if (!imageLoaded) { 569 writeLock(); 570 try { 571 while (!imageLoaded) { 572 try { 573 cond.await(5000, TimeUnit.MILLISECONDS); 574 } catch (InterruptedException ignored) { 575 } 576 } 577 } finally { 578 writeUnlock(); 579 } 580 } 581 } 582 583 /** 584 * Clear all loaded data 585 */ 586 void clear() { 587 dir.reset(); 588 dtSecretManager.reset(); 589 blockIdManager.clear(); 590 leaseManager.removeAllLeases(); 591 snapshotManager.clearSnapshottableDirs(); 592 cacheManager.clear(); 593 setImageLoaded(false); 594 blockManager.clear(); 595 } 596 597 @VisibleForTesting 598 LeaseManager getLeaseManager() { 599 return leaseManager; 600 } 601 602 boolean isHaEnabled() { 603 return haEnabled; 604 } 605 606 /** 607 * Check the supplied configuration for correctness. 608 * @param conf Supplies the configuration to validate. 609 * @throws IOException if the configuration could not be queried. 610 * @throws IllegalArgumentException if the configuration is invalid. 611 */ 612 private static void checkConfiguration(Configuration conf) 613 throws IOException { 614 615 final Collection<URI> namespaceDirs = 616 FSNamesystem.getNamespaceDirs(conf); 617 final Collection<URI> editsDirs = 618 FSNamesystem.getNamespaceEditsDirs(conf); 619 final Collection<URI> requiredEditsDirs = 620 FSNamesystem.getRequiredNamespaceEditsDirs(conf); 621 final Collection<URI> sharedEditsDirs = 622 FSNamesystem.getSharedEditsDirs(conf); 623 624 for (URI u : requiredEditsDirs) { 625 if (u.toString().compareTo( 626 DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_DEFAULT) == 0) { 627 continue; 628 } 629 630 // Each required directory must also be in editsDirs or in 631 // sharedEditsDirs. 632 if (!editsDirs.contains(u) && 633 !sharedEditsDirs.contains(u)) { 634 throw new IllegalArgumentException( 635 "Required edits directory " + u.toString() + " not present in " + 636 DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY + ". " + 637 DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY + "=" + 638 editsDirs.toString() + "; " + 639 DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY + "=" + 640 requiredEditsDirs.toString() + ". " + 641 DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY + "=" + 642 sharedEditsDirs.toString() + "."); 643 } 644 } 645 646 if (namespaceDirs.size() == 1) { 647 LOG.warn("Only one image storage directory (" 648 + DFS_NAMENODE_NAME_DIR_KEY + ") configured. Beware of data loss" 649 + " due to lack of redundant storage directories!"); 650 } 651 if (editsDirs.size() == 1) { 652 LOG.warn("Only one namespace edits storage directory (" 653 + DFS_NAMENODE_EDITS_DIR_KEY + ") configured. Beware of data loss" 654 + " due to lack of redundant storage directories!"); 655 } 656 } 657 658 /** 659 * Instantiates an FSNamesystem loaded from the image and edits 660 * directories specified in the passed Configuration. 661 * 662 * @param conf the Configuration which specifies the storage directories 663 * from which to load 664 * @return an FSNamesystem which contains the loaded namespace 665 * @throws IOException if loading fails 666 */ 667 static FSNamesystem loadFromDisk(Configuration conf) throws IOException { 668 669 checkConfiguration(conf); 670 FSImage fsImage = new FSImage(conf, 671 FSNamesystem.getNamespaceDirs(conf), 672 FSNamesystem.getNamespaceEditsDirs(conf)); 673 FSNamesystem namesystem = new FSNamesystem(conf, fsImage, false); 674 StartupOption startOpt = NameNode.getStartupOption(conf); 675 if (startOpt == StartupOption.RECOVER) { 676 namesystem.setSafeMode(SafeModeAction.SAFEMODE_ENTER); 677 } 678 679 long loadStart = monotonicNow(); 680 try { 681 namesystem.loadFSImage(startOpt); 682 } catch (IOException ioe) { 683 LOG.warn("Encountered exception loading fsimage", ioe); 684 fsImage.close(); 685 throw ioe; 686 } 687 long timeTakenToLoadFSImage = monotonicNow() - loadStart; 688 LOG.info("Finished loading FSImage in " + timeTakenToLoadFSImage + " msecs"); 689 NameNodeMetrics nnMetrics = NameNode.getNameNodeMetrics(); 690 if (nnMetrics != null) { 691 nnMetrics.setFsImageLoadTime((int) timeTakenToLoadFSImage); 692 } 693 return namesystem; 694 } 695 696 FSNamesystem(Configuration conf, FSImage fsImage) throws IOException { 697 this(conf, fsImage, false); 698 } 699 700 /** 701 * Create an FSNamesystem associated with the specified image. 702 * 703 * Note that this does not load any data off of disk -- if you would 704 * like that behavior, use {@link #loadFromDisk(Configuration)} 705 * 706 * @param conf configuration 707 * @param fsImage The FSImage to associate with 708 * @param ignoreRetryCache Whether or not should ignore the retry cache setup 709 * step. For Secondary NN this should be set to true. 710 * @throws IOException on bad configuration 711 */ 712 FSNamesystem(Configuration conf, FSImage fsImage, boolean ignoreRetryCache) 713 throws IOException { 714 provider = DFSUtil.createKeyProviderCryptoExtension(conf); 715 if (provider == null) { 716 LOG.info("No KeyProvider found."); 717 } else { 718 LOG.info("Found KeyProvider: " + provider.toString()); 719 } 720 if (conf.getBoolean(DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY, 721 DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT)) { 722 LOG.info("Enabling async auditlog"); 723 enableAsyncAuditLog(); 724 } 725 boolean fair = conf.getBoolean("dfs.namenode.fslock.fair", true); 726 LOG.info("fsLock is fair:" + fair); 727 fsLock = new FSNamesystemLock(fair); 728 cond = fsLock.writeLock().newCondition(); 729 cpLock = new ReentrantLock(); 730 731 this.fsImage = fsImage; 732 try { 733 resourceRecheckInterval = conf.getLong( 734 DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY, 735 DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_DEFAULT); 736 737 this.blockManager = new BlockManager(this, conf); 738 this.datanodeStatistics = blockManager.getDatanodeManager().getDatanodeStatistics(); 739 this.blockIdManager = new BlockIdManager(blockManager); 740 741 this.fsOwner = UserGroupInformation.getCurrentUser(); 742 this.supergroup = conf.get(DFS_PERMISSIONS_SUPERUSERGROUP_KEY, 743 DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT); 744 this.isPermissionEnabled = conf.getBoolean(DFS_PERMISSIONS_ENABLED_KEY, 745 DFS_PERMISSIONS_ENABLED_DEFAULT); 746 LOG.info("fsOwner = " + fsOwner); 747 LOG.info("supergroup = " + supergroup); 748 LOG.info("isPermissionEnabled = " + isPermissionEnabled); 749 750 // block allocation has to be persisted in HA using a shared edits directory 751 // so that the standby has up-to-date namespace information 752 nameserviceId = DFSUtil.getNamenodeNameServiceId(conf); 753 this.haEnabled = HAUtil.isHAEnabled(conf, nameserviceId); 754 755 // Sanity check the HA-related config. 756 if (nameserviceId != null) { 757 LOG.info("Determined nameservice ID: " + nameserviceId); 758 } 759 LOG.info("HA Enabled: " + haEnabled); 760 if (!haEnabled && HAUtil.usesSharedEditsDir(conf)) { 761 LOG.warn("Configured NNs:\n" + DFSUtil.nnAddressesAsString(conf)); 762 throw new IOException("Invalid configuration: a shared edits dir " + 763 "must not be specified if HA is not enabled."); 764 } 765 766 // Get the checksum type from config 767 String checksumTypeStr = conf.get(DFS_CHECKSUM_TYPE_KEY, DFS_CHECKSUM_TYPE_DEFAULT); 768 DataChecksum.Type checksumType; 769 try { 770 checksumType = DataChecksum.Type.valueOf(checksumTypeStr); 771 } catch (IllegalArgumentException iae) { 772 throw new IOException("Invalid checksum type in " 773 + DFS_CHECKSUM_TYPE_KEY + ": " + checksumTypeStr); 774 } 775 776 this.serverDefaults = new FsServerDefaults( 777 conf.getLongBytes(DFS_BLOCK_SIZE_KEY, DFS_BLOCK_SIZE_DEFAULT), 778 conf.getInt(DFS_BYTES_PER_CHECKSUM_KEY, DFS_BYTES_PER_CHECKSUM_DEFAULT), 779 conf.getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY, DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT), 780 (short) conf.getInt(DFS_REPLICATION_KEY, DFS_REPLICATION_DEFAULT), 781 conf.getInt(IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT), 782 conf.getBoolean(DFS_ENCRYPT_DATA_TRANSFER_KEY, DFS_ENCRYPT_DATA_TRANSFER_DEFAULT), 783 conf.getLong(FS_TRASH_INTERVAL_KEY, FS_TRASH_INTERVAL_DEFAULT), 784 checksumType); 785 786 this.maxFsObjects = conf.getLong(DFS_NAMENODE_MAX_OBJECTS_KEY, 787 DFS_NAMENODE_MAX_OBJECTS_DEFAULT); 788 789 this.minBlockSize = conf.getLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, 790 DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_DEFAULT); 791 this.maxBlocksPerFile = conf.getLong(DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY, 792 DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT); 793 this.accessTimePrecision = conf.getLong(DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 794 DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT); 795 this.supportAppends = conf.getBoolean(DFS_SUPPORT_APPEND_KEY, DFS_SUPPORT_APPEND_DEFAULT); 796 LOG.info("Append Enabled: " + supportAppends); 797 798 this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf); 799 800 this.standbyShouldCheckpoint = conf.getBoolean( 801 DFS_HA_STANDBY_CHECKPOINTS_KEY, DFS_HA_STANDBY_CHECKPOINTS_DEFAULT); 802 // # edit autoroll threshold is a multiple of the checkpoint threshold 803 this.editLogRollerThreshold = (long) 804 (conf.getFloat( 805 DFS_NAMENODE_EDIT_LOG_AUTOROLL_MULTIPLIER_THRESHOLD, 806 DFS_NAMENODE_EDIT_LOG_AUTOROLL_MULTIPLIER_THRESHOLD_DEFAULT) * 807 conf.getLong( 808 DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 809 DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT)); 810 this.editLogRollerInterval = conf.getInt( 811 DFS_NAMENODE_EDIT_LOG_AUTOROLL_CHECK_INTERVAL_MS, 812 DFS_NAMENODE_EDIT_LOG_AUTOROLL_CHECK_INTERVAL_MS_DEFAULT); 813 814 this.lazyPersistFileScrubIntervalSec = conf.getInt( 815 DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC, 816 DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC_DEFAULT); 817 818 if (this.lazyPersistFileScrubIntervalSec == 0) { 819 throw new IllegalArgumentException( 820 DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC + " must be non-zero."); 821 } 822 823 // For testing purposes, allow the DT secret manager to be started regardless 824 // of whether security is enabled. 825 alwaysUseDelegationTokensForTests = conf.getBoolean( 826 DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, 827 DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT); 828 829 this.dtSecretManager = createDelegationTokenSecretManager(conf); 830 this.dir = new FSDirectory(this, conf); 831 this.snapshotManager = new SnapshotManager(dir); 832 this.cacheManager = new CacheManager(this, conf, blockManager); 833 this.safeMode = new SafeModeInfo(conf); 834 this.topConf = new TopConf(conf); 835 this.auditLoggers = initAuditLoggers(conf); 836 this.isDefaultAuditLogger = auditLoggers.size() == 1 && 837 auditLoggers.get(0) instanceof DefaultAuditLogger; 838 this.retryCache = ignoreRetryCache ? null : initRetryCache(conf); 839 Class<? extends INodeAttributeProvider> klass = conf.getClass( 840 DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY, 841 null, INodeAttributeProvider.class); 842 if (klass != null) { 843 inodeAttributeProvider = ReflectionUtils.newInstance(klass, conf); 844 LOG.info("Using INode attribute provider: " + klass.getName()); 845 } 846 } catch(IOException e) { 847 LOG.error(getClass().getSimpleName() + " initialization failed.", e); 848 close(); 849 throw e; 850 } catch (RuntimeException re) { 851 LOG.error(getClass().getSimpleName() + " initialization failed.", re); 852 close(); 853 throw re; 854 } 855 } 856 857 @VisibleForTesting 858 public List<AuditLogger> getAuditLoggers() { 859 return auditLoggers; 860 } 861 862 @VisibleForTesting 863 public RetryCache getRetryCache() { 864 return retryCache; 865 } 866 867 void lockRetryCache() { 868 if (retryCache != null) { 869 retryCache.lock(); 870 } 871 } 872 873 void unlockRetryCache() { 874 if (retryCache != null) { 875 retryCache.unlock(); 876 } 877 } 878 879 /** Whether or not retry cache is enabled */ 880 boolean hasRetryCache() { 881 return retryCache != null; 882 } 883 884 void addCacheEntryWithPayload(byte[] clientId, int callId, Object payload) { 885 if (retryCache != null) { 886 retryCache.addCacheEntryWithPayload(clientId, callId, payload); 887 } 888 } 889 890 void addCacheEntry(byte[] clientId, int callId) { 891 if (retryCache != null) { 892 retryCache.addCacheEntry(clientId, callId); 893 } 894 } 895 896 @VisibleForTesting 897 public KeyProviderCryptoExtension getProvider() { 898 return provider; 899 } 900 901 @VisibleForTesting 902 static RetryCache initRetryCache(Configuration conf) { 903 boolean enable = conf.getBoolean(DFS_NAMENODE_ENABLE_RETRY_CACHE_KEY, 904 DFS_NAMENODE_ENABLE_RETRY_CACHE_DEFAULT); 905 LOG.info("Retry cache on namenode is " + (enable ? "enabled" : "disabled")); 906 if (enable) { 907 float heapPercent = conf.getFloat( 908 DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_KEY, 909 DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_DEFAULT); 910 long entryExpiryMillis = conf.getLong( 911 DFS_NAMENODE_RETRY_CACHE_EXPIRYTIME_MILLIS_KEY, 912 DFS_NAMENODE_RETRY_CACHE_EXPIRYTIME_MILLIS_DEFAULT); 913 LOG.info("Retry cache will use " + heapPercent 914 + " of total heap and retry cache entry expiry time is " 915 + entryExpiryMillis + " millis"); 916 long entryExpiryNanos = entryExpiryMillis * 1000 * 1000; 917 return new RetryCache("NameNodeRetryCache", heapPercent, 918 entryExpiryNanos); 919 } 920 return null; 921 } 922 923 private List<AuditLogger> initAuditLoggers(Configuration conf) { 924 // Initialize the custom access loggers if configured. 925 Collection<String> alClasses = conf.getStringCollection(DFS_NAMENODE_AUDIT_LOGGERS_KEY); 926 List<AuditLogger> auditLoggers = Lists.newArrayList(); 927 if (alClasses != null && !alClasses.isEmpty()) { 928 for (String className : alClasses) { 929 try { 930 AuditLogger logger; 931 if (DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME.equals(className)) { 932 logger = new DefaultAuditLogger(); 933 } else { 934 logger = (AuditLogger) Class.forName(className).newInstance(); 935 } 936 logger.initialize(conf); 937 auditLoggers.add(logger); 938 } catch (RuntimeException re) { 939 throw re; 940 } catch (Exception e) { 941 throw new RuntimeException(e); 942 } 943 } 944 } 945 946 // Make sure there is at least one logger installed. 947 if (auditLoggers.isEmpty()) { 948 auditLoggers.add(new DefaultAuditLogger()); 949 } 950 951 // Add audit logger to calculate top users 952 if (topConf.isEnabled) { 953 topMetrics = new TopMetrics(conf, topConf.nntopReportingPeriodsMs); 954 auditLoggers.add(new TopAuditLogger(topMetrics)); 955 } 956 957 return Collections.unmodifiableList(auditLoggers); 958 } 959 960 private void loadFSImage(StartupOption startOpt) throws IOException { 961 final FSImage fsImage = getFSImage(); 962 963 // format before starting up if requested 964 if (startOpt == StartupOption.FORMAT) { 965 966 fsImage.format(this, fsImage.getStorage().determineClusterId());// reuse current id 967 968 startOpt = StartupOption.REGULAR; 969 } 970 boolean success = false; 971 writeLock(); 972 try { 973 // We shouldn't be calling saveNamespace if we've come up in standby state. 974 MetaRecoveryContext recovery = startOpt.createRecoveryContext(); 975 final boolean staleImage 976 = fsImage.recoverTransitionRead(startOpt, this, recovery); 977 if (RollingUpgradeStartupOption.ROLLBACK.matches(startOpt) || 978 RollingUpgradeStartupOption.DOWNGRADE.matches(startOpt)) { 979 rollingUpgradeInfo = null; 980 } 981 final boolean needToSave = staleImage && !haEnabled && !isRollingUpgrade(); 982 LOG.info("Need to save fs image? " + needToSave 983 + " (staleImage=" + staleImage + ", haEnabled=" + haEnabled 984 + ", isRollingUpgrade=" + isRollingUpgrade() + ")"); 985 if (needToSave) { 986 fsImage.saveNamespace(this); 987 } else { 988 updateStorageVersionForRollingUpgrade(fsImage.getLayoutVersion(), 989 startOpt); 990 // No need to save, so mark the phase done. 991 StartupProgress prog = NameNode.getStartupProgress(); 992 prog.beginPhase(Phase.SAVING_CHECKPOINT); 993 prog.endPhase(Phase.SAVING_CHECKPOINT); 994 } 995 // This will start a new log segment and write to the seen_txid file, so 996 // we shouldn't do it when coming up in standby state 997 if (!haEnabled || (haEnabled && startOpt == StartupOption.UPGRADE) 998 || (haEnabled && startOpt == StartupOption.UPGRADEONLY)) { 999 fsImage.openEditLogForWrite(); 1000 } 1001 success = true; 1002 } finally { 1003 if (!success) { 1004 fsImage.close(); 1005 } 1006 writeUnlock(); 1007 } 1008 imageLoadComplete(); 1009 } 1010 1011 private void updateStorageVersionForRollingUpgrade(final long layoutVersion, 1012 StartupOption startOpt) throws IOException { 1013 boolean rollingStarted = RollingUpgradeStartupOption.STARTED 1014 .matches(startOpt) && layoutVersion > HdfsConstants 1015 .NAMENODE_LAYOUT_VERSION; 1016 boolean rollingRollback = RollingUpgradeStartupOption.ROLLBACK 1017 .matches(startOpt); 1018 if (rollingRollback || rollingStarted) { 1019 fsImage.updateStorageVersion(); 1020 } 1021 } 1022 1023 private void startSecretManager() { 1024 if (dtSecretManager != null) { 1025 try { 1026 dtSecretManager.startThreads(); 1027 } catch (IOException e) { 1028 // Inability to start secret manager 1029 // can't be recovered from. 1030 throw new RuntimeException(e); 1031 } 1032 } 1033 } 1034 1035 private void startSecretManagerIfNecessary() { 1036 boolean shouldRun = shouldUseDelegationTokens() && 1037 !isInSafeMode() && getEditLog().isOpenForWrite(); 1038 boolean running = dtSecretManager.isRunning(); 1039 if (shouldRun && !running) { 1040 startSecretManager(); 1041 } 1042 } 1043 1044 private void stopSecretManager() { 1045 if (dtSecretManager != null) { 1046 dtSecretManager.stopThreads(); 1047 } 1048 } 1049 1050 /** 1051 * Start services common to both active and standby states 1052 */ 1053 void startCommonServices(Configuration conf, HAContext haContext) throws IOException { 1054 this.registerMBean(); // register the MBean for the FSNamesystemState 1055 writeLock(); 1056 this.haContext = haContext; 1057 try { 1058 nnResourceChecker = new NameNodeResourceChecker(conf); 1059 checkAvailableResources(); 1060 assert safeMode != null && !isPopulatingReplQueues(); 1061 StartupProgress prog = NameNode.getStartupProgress(); 1062 prog.beginPhase(Phase.SAFEMODE); 1063 prog.setTotal(Phase.SAFEMODE, STEP_AWAITING_REPORTED_BLOCKS, 1064 getCompleteBlocksTotal()); 1065 setBlockTotal(); 1066 blockManager.activate(conf); 1067 } finally { 1068 writeUnlock(); 1069 } 1070 1071 registerMXBean(); 1072 DefaultMetricsSystem.instance().register(this); 1073 if (inodeAttributeProvider != null) { 1074 inodeAttributeProvider.start(); 1075 dir.setINodeAttributeProvider(inodeAttributeProvider); 1076 } 1077 snapshotManager.registerMXBean(); 1078 } 1079 1080 /** 1081 * Stop services common to both active and standby states 1082 */ 1083 void stopCommonServices() { 1084 writeLock(); 1085 if (inodeAttributeProvider != null) { 1086 dir.setINodeAttributeProvider(null); 1087 inodeAttributeProvider.stop(); 1088 } 1089 try { 1090 if (blockManager != null) blockManager.close(); 1091 } finally { 1092 writeUnlock(); 1093 } 1094 RetryCache.clear(retryCache); 1095 } 1096 1097 /** 1098 * Start services required in active state 1099 * @throws IOException 1100 */ 1101 void startActiveServices() throws IOException { 1102 startingActiveService = true; 1103 LOG.info("Starting services required for active state"); 1104 writeLock(); 1105 try { 1106 FSEditLog editLog = getFSImage().getEditLog(); 1107 1108 if (!editLog.isOpenForWrite()) { 1109 // During startup, we're already open for write during initialization. 1110 editLog.initJournalsForWrite(); 1111 // May need to recover 1112 editLog.recoverUnclosedStreams(); 1113 1114 LOG.info("Catching up to latest edits from old active before " + 1115 "taking over writer role in edits logs"); 1116 editLogTailer.catchupDuringFailover(); 1117 1118 blockManager.setPostponeBlocksFromFuture(false); 1119 blockManager.getDatanodeManager().markAllDatanodesStale(); 1120 blockManager.clearQueues(); 1121 blockManager.processAllPendingDNMessages(); 1122 1123 // Only need to re-process the queue, If not in SafeMode. 1124 if (!isInSafeMode()) { 1125 LOG.info("Reprocessing replication and invalidation queues"); 1126 initializeReplQueues(); 1127 } 1128 1129 if (LOG.isDebugEnabled()) { 1130 LOG.debug("NameNode metadata after re-processing " + 1131 "replication and invalidation queues during failover:\n" + 1132 metaSaveAsString()); 1133 } 1134 1135 long nextTxId = getFSImage().getLastAppliedTxId() + 1; 1136 LOG.info("Will take over writing edit logs at txnid " + 1137 nextTxId); 1138 editLog.setNextTxId(nextTxId); 1139 1140 getFSImage().editLog.openForWrite(); 1141 } 1142 1143 // Enable quota checks. 1144 dir.enableQuotaChecks(); 1145 if (haEnabled) { 1146 // Renew all of the leases before becoming active. 1147 // This is because, while we were in standby mode, 1148 // the leases weren't getting renewed on this NN. 1149 // Give them all a fresh start here. 1150 leaseManager.renewAllLeases(); 1151 } 1152 leaseManager.startMonitor(); 1153 startSecretManagerIfNecessary(); 1154 1155 //ResourceMonitor required only at ActiveNN. See HDFS-2914 1156 this.nnrmthread = new Daemon(new NameNodeResourceMonitor()); 1157 nnrmthread.start(); 1158 1159 nnEditLogRoller = new Daemon(new NameNodeEditLogRoller( 1160 editLogRollerThreshold, editLogRollerInterval)); 1161 nnEditLogRoller.start(); 1162 1163 if (lazyPersistFileScrubIntervalSec > 0) { 1164 lazyPersistFileScrubber = new Daemon(new LazyPersistFileScrubber( 1165 lazyPersistFileScrubIntervalSec)); 1166 lazyPersistFileScrubber.start(); 1167 } 1168 1169 cacheManager.startMonitorThread(); 1170 blockManager.getDatanodeManager().setShouldSendCachingCommands(true); 1171 } finally { 1172 startingActiveService = false; 1173 checkSafeMode(); 1174 writeUnlock(); 1175 } 1176 } 1177 1178 /** 1179 * Initialize replication queues. 1180 */ 1181 private void initializeReplQueues() { 1182 LOG.info("initializing replication queues"); 1183 blockManager.processMisReplicatedBlocks(); 1184 initializedReplQueues = true; 1185 } 1186 1187 private boolean inActiveState() { 1188 return haContext != null && 1189 haContext.getState().getServiceState() == HAServiceState.ACTIVE; 1190 } 1191 1192 /** 1193 * @return Whether the namenode is transitioning to active state and is in the 1194 * middle of the {@link #startActiveServices()} 1195 */ 1196 public boolean inTransitionToActive() { 1197 return haEnabled && inActiveState() && startingActiveService; 1198 } 1199 1200 private boolean shouldUseDelegationTokens() { 1201 return UserGroupInformation.isSecurityEnabled() || 1202 alwaysUseDelegationTokensForTests; 1203 } 1204 1205 /** 1206 * Stop services required in active state 1207 */ 1208 void stopActiveServices() { 1209 LOG.info("Stopping services started for active state"); 1210 writeLock(); 1211 try { 1212 stopSecretManager(); 1213 leaseManager.stopMonitor(); 1214 if (nnrmthread != null) { 1215 ((NameNodeResourceMonitor) nnrmthread.getRunnable()).stopMonitor(); 1216 nnrmthread.interrupt(); 1217 } 1218 if (nnEditLogRoller != null) { 1219 ((NameNodeEditLogRoller)nnEditLogRoller.getRunnable()).stop(); 1220 nnEditLogRoller.interrupt(); 1221 } 1222 if (lazyPersistFileScrubber != null) { 1223 ((LazyPersistFileScrubber) lazyPersistFileScrubber.getRunnable()).stop(); 1224 lazyPersistFileScrubber.interrupt(); 1225 } 1226 if (dir != null && getFSImage() != null) { 1227 if (getFSImage().editLog != null) { 1228 getFSImage().editLog.close(); 1229 } 1230 // Update the fsimage with the last txid that we wrote 1231 // so that the tailer starts from the right spot. 1232 getFSImage().updateLastAppliedTxIdFromWritten(); 1233 } 1234 if (cacheManager != null) { 1235 cacheManager.stopMonitorThread(); 1236 cacheManager.clearDirectiveStats(); 1237 } 1238 blockManager.getDatanodeManager().clearPendingCachingCommands(); 1239 blockManager.getDatanodeManager().setShouldSendCachingCommands(false); 1240 // Don't want to keep replication queues when not in Active. 1241 blockManager.clearQueues(); 1242 initializedReplQueues = false; 1243 } finally { 1244 writeUnlock(); 1245 } 1246 } 1247 1248 /** 1249 * Start services required in standby state 1250 * 1251 * @throws IOException 1252 */ 1253 void startStandbyServices(final Configuration conf) throws IOException { 1254 LOG.info("Starting services required for standby state"); 1255 if (!getFSImage().editLog.isOpenForRead()) { 1256 // During startup, we're already open for read. 1257 getFSImage().editLog.initSharedJournalsForRead(); 1258 } 1259 1260 blockManager.setPostponeBlocksFromFuture(true); 1261 1262 // Disable quota checks while in standby. 1263 dir.disableQuotaChecks(); 1264 editLogTailer = new EditLogTailer(this, conf); 1265 editLogTailer.start(); 1266 if (standbyShouldCheckpoint) { 1267 standbyCheckpointer = new StandbyCheckpointer(conf, this); 1268 standbyCheckpointer.start(); 1269 } 1270 } 1271 1272 /** 1273 * Called when the NN is in Standby state and the editlog tailer tails the 1274 * OP_ROLLING_UPGRADE_START. 1275 */ 1276 void triggerRollbackCheckpoint() { 1277 setNeedRollbackFsImage(true); 1278 if (standbyCheckpointer != null) { 1279 standbyCheckpointer.triggerRollbackCheckpoint(); 1280 } 1281 } 1282 1283 /** 1284 * Called while the NN is in Standby state, but just about to be 1285 * asked to enter Active state. This cancels any checkpoints 1286 * currently being taken. 1287 */ 1288 void prepareToStopStandbyServices() throws ServiceFailedException { 1289 if (standbyCheckpointer != null) { 1290 standbyCheckpointer.cancelAndPreventCheckpoints( 1291 "About to leave standby state"); 1292 } 1293 } 1294 1295 /** Stop services required in standby state */ 1296 void stopStandbyServices() throws IOException { 1297 LOG.info("Stopping services started for standby state"); 1298 if (standbyCheckpointer != null) { 1299 standbyCheckpointer.stop(); 1300 } 1301 if (editLogTailer != null) { 1302 editLogTailer.stop(); 1303 } 1304 if (dir != null && getFSImage() != null && getFSImage().editLog != null) { 1305 getFSImage().editLog.close(); 1306 } 1307 } 1308 1309 @Override 1310 public void checkOperation(OperationCategory op) throws StandbyException { 1311 if (haContext != null) { 1312 // null in some unit tests 1313 haContext.checkOperation(op); 1314 } 1315 } 1316 1317 /** 1318 * @throws RetriableException 1319 * If 1) The NameNode is in SafeMode, 2) HA is enabled, and 3) 1320 * NameNode is in active state 1321 * @throws SafeModeException 1322 * Otherwise if NameNode is in SafeMode. 1323 */ 1324 void checkNameNodeSafeMode(String errorMsg) 1325 throws RetriableException, SafeModeException { 1326 if (isInSafeMode()) { 1327 SafeModeException se = new SafeModeException(errorMsg, safeMode); 1328 if (haEnabled && haContext != null 1329 && haContext.getState().getServiceState() == HAServiceState.ACTIVE 1330 && shouldRetrySafeMode(this.safeMode)) { 1331 throw new RetriableException(se); 1332 } else { 1333 throw se; 1334 } 1335 } 1336 } 1337 1338 boolean isPermissionEnabled() { 1339 return isPermissionEnabled; 1340 } 1341 1342 /** 1343 * We already know that the safemode is on. We will throw a RetriableException 1344 * if the safemode is not manual or caused by low resource. 1345 */ 1346 private boolean shouldRetrySafeMode(SafeModeInfo safeMode) { 1347 if (safeMode == null) { 1348 return false; 1349 } else { 1350 return !safeMode.isManual() && !safeMode.areResourcesLow(); 1351 } 1352 } 1353 1354 public static Collection<URI> getNamespaceDirs(Configuration conf) { 1355 return getStorageDirs(conf, DFS_NAMENODE_NAME_DIR_KEY); 1356 } 1357 1358 /** 1359 * Get all edits dirs which are required. If any shared edits dirs are 1360 * configured, these are also included in the set of required dirs. 1361 * 1362 * @param conf the HDFS configuration. 1363 * @return all required dirs. 1364 */ 1365 public static Collection<URI> getRequiredNamespaceEditsDirs(Configuration conf) { 1366 Set<URI> ret = new HashSet<URI>(); 1367 ret.addAll(getStorageDirs(conf, DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY)); 1368 ret.addAll(getSharedEditsDirs(conf)); 1369 return ret; 1370 } 1371 1372 private static Collection<URI> getStorageDirs(Configuration conf, 1373 String propertyName) { 1374 Collection<String> dirNames = conf.getTrimmedStringCollection(propertyName); 1375 StartupOption startOpt = NameNode.getStartupOption(conf); 1376 if(startOpt == StartupOption.IMPORT) { 1377 // In case of IMPORT this will get rid of default directories 1378 // but will retain directories specified in hdfs-site.xml 1379 // When importing image from a checkpoint, the name-node can 1380 // start with empty set of storage directories. 1381 Configuration cE = new HdfsConfiguration(false); 1382 cE.addResource("core-default.xml"); 1383 cE.addResource("core-site.xml"); 1384 cE.addResource("hdfs-default.xml"); 1385 Collection<String> dirNames2 = cE.getTrimmedStringCollection(propertyName); 1386 dirNames.removeAll(dirNames2); 1387 if(dirNames.isEmpty()) 1388 LOG.warn("!!! WARNING !!!" + 1389 "\n\tThe NameNode currently runs without persistent storage." + 1390 "\n\tAny changes to the file system meta-data may be lost." + 1391 "\n\tRecommended actions:" + 1392 "\n\t\t- shutdown and restart NameNode with configured \"" 1393 + propertyName + "\" in hdfs-site.xml;" + 1394 "\n\t\t- use Backup Node as a persistent and up-to-date storage " + 1395 "of the file system meta-data."); 1396 } else if (dirNames.isEmpty()) { 1397 dirNames = Collections.singletonList( 1398 DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_DEFAULT); 1399 } 1400 return Util.stringCollectionAsURIs(dirNames); 1401 } 1402 1403 /** 1404 * Return an ordered list of edits directories to write to. 1405 * The list is ordered such that all shared edits directories 1406 * are ordered before non-shared directories, and any duplicates 1407 * are removed. The order they are specified in the configuration 1408 * is retained. 1409 * @return Collection of shared edits directories. 1410 * @throws IOException if multiple shared edits directories are configured 1411 */ 1412 public static List<URI> getNamespaceEditsDirs(Configuration conf) 1413 throws IOException { 1414 return getNamespaceEditsDirs(conf, true); 1415 } 1416 1417 public static List<URI> getNamespaceEditsDirs(Configuration conf, 1418 boolean includeShared) 1419 throws IOException { 1420 // Use a LinkedHashSet so that order is maintained while we de-dup 1421 // the entries. 1422 LinkedHashSet<URI> editsDirs = new LinkedHashSet<URI>(); 1423 1424 if (includeShared) { 1425 List<URI> sharedDirs = getSharedEditsDirs(conf); 1426 1427 // Fail until multiple shared edits directories are supported (HDFS-2782) 1428 if (sharedDirs.size() > 1) { 1429 throw new IOException( 1430 "Multiple shared edits directories are not yet supported"); 1431 } 1432 1433 // First add the shared edits dirs. It's critical that the shared dirs 1434 // are added first, since JournalSet syncs them in the order they are listed, 1435 // and we need to make sure all edits are in place in the shared storage 1436 // before they are replicated locally. See HDFS-2874. 1437 for (URI dir : sharedDirs) { 1438 if (!editsDirs.add(dir)) { 1439 LOG.warn("Edits URI " + dir + " listed multiple times in " + 1440 DFS_NAMENODE_SHARED_EDITS_DIR_KEY + ". Ignoring duplicates."); 1441 } 1442 } 1443 } 1444 // Now add the non-shared dirs. 1445 for (URI dir : getStorageDirs(conf, DFS_NAMENODE_EDITS_DIR_KEY)) { 1446 if (!editsDirs.add(dir)) { 1447 LOG.warn("Edits URI " + dir + " listed multiple times in " + 1448 DFS_NAMENODE_SHARED_EDITS_DIR_KEY + " and " + 1449 DFS_NAMENODE_EDITS_DIR_KEY + ". Ignoring duplicates."); 1450 } 1451 } 1452 1453 if (editsDirs.isEmpty()) { 1454 // If this is the case, no edit dirs have been explicitly configured. 1455 // Image dirs are to be used for edits too. 1456 return Lists.newArrayList(getNamespaceDirs(conf)); 1457 } else { 1458 return Lists.newArrayList(editsDirs); 1459 } 1460 } 1461 1462 /** 1463 * Returns edit directories that are shared between primary and secondary. 1464 * @param conf configuration 1465 * @return collection of edit directories from {@code conf} 1466 */ 1467 public static List<URI> getSharedEditsDirs(Configuration conf) { 1468 // don't use getStorageDirs here, because we want an empty default 1469 // rather than the dir in /tmp 1470 Collection<String> dirNames = conf.getTrimmedStringCollection( 1471 DFS_NAMENODE_SHARED_EDITS_DIR_KEY); 1472 return Util.stringCollectionAsURIs(dirNames); 1473 } 1474 1475 @Override 1476 public void readLock() { 1477 this.fsLock.readLock().lock(); 1478 } 1479 @Override 1480 public void readUnlock() { 1481 this.fsLock.readLock().unlock(); 1482 } 1483 @Override 1484 public void writeLock() { 1485 this.fsLock.writeLock().lock(); 1486 } 1487 @Override 1488 public void writeLockInterruptibly() throws InterruptedException { 1489 this.fsLock.writeLock().lockInterruptibly(); 1490 } 1491 @Override 1492 public void writeUnlock() { 1493 this.fsLock.writeLock().unlock(); 1494 } 1495 @Override 1496 public boolean hasWriteLock() { 1497 return this.fsLock.isWriteLockedByCurrentThread(); 1498 } 1499 @Override 1500 public boolean hasReadLock() { 1501 return this.fsLock.getReadHoldCount() > 0 || hasWriteLock(); 1502 } 1503 1504 public int getReadHoldCount() { 1505 return this.fsLock.getReadHoldCount(); 1506 } 1507 1508 public int getWriteHoldCount() { 1509 return this.fsLock.getWriteHoldCount(); 1510 } 1511 1512 /** Lock the checkpoint lock */ 1513 public void cpLock() { 1514 this.cpLock.lock(); 1515 } 1516 1517 /** Lock the checkpoint lock interrupibly */ 1518 public void cpLockInterruptibly() throws InterruptedException { 1519 this.cpLock.lockInterruptibly(); 1520 } 1521 1522 /** Unlock the checkpoint lock */ 1523 public void cpUnlock() { 1524 this.cpLock.unlock(); 1525 } 1526 1527 1528 NamespaceInfo getNamespaceInfo() { 1529 readLock(); 1530 try { 1531 return unprotectedGetNamespaceInfo(); 1532 } finally { 1533 readUnlock(); 1534 } 1535 } 1536 1537 /** 1538 * Version of @see #getNamespaceInfo() that is not protected by a lock. 1539 */ 1540 NamespaceInfo unprotectedGetNamespaceInfo() { 1541 return new NamespaceInfo(getFSImage().getStorage().getNamespaceID(), 1542 getClusterId(), getBlockPoolId(), 1543 getFSImage().getStorage().getCTime()); 1544 } 1545 1546 /** 1547 * Close down this file system manager. 1548 * Causes heartbeat and lease daemons to stop; waits briefly for 1549 * them to finish, but a short timeout returns control back to caller. 1550 */ 1551 void close() { 1552 fsRunning = false; 1553 try { 1554 stopCommonServices(); 1555 if (smmthread != null) smmthread.interrupt(); 1556 } finally { 1557 // using finally to ensure we also wait for lease daemon 1558 try { 1559 stopActiveServices(); 1560 stopStandbyServices(); 1561 } catch (IOException ie) { 1562 } finally { 1563 IOUtils.cleanup(LOG, dir); 1564 IOUtils.cleanup(LOG, fsImage); 1565 } 1566 } 1567 } 1568 1569 @Override 1570 public boolean isRunning() { 1571 return fsRunning; 1572 } 1573 1574 @Override 1575 public boolean isInStandbyState() { 1576 if (haContext == null || haContext.getState() == null) { 1577 // We're still starting up. In this case, if HA is 1578 // on for the cluster, we always start in standby. Otherwise 1579 // start in active. 1580 return haEnabled; 1581 } 1582 1583 return HAServiceState.STANDBY == haContext.getState().getServiceState(); 1584 } 1585 1586 /** 1587 * Dump all metadata into specified file 1588 */ 1589 void metaSave(String filename) throws IOException { 1590 checkSuperuserPrivilege(); 1591 checkOperation(OperationCategory.UNCHECKED); 1592 writeLock(); 1593 try { 1594 checkOperation(OperationCategory.UNCHECKED); 1595 File file = new File(System.getProperty("hadoop.log.dir"), filename); 1596 PrintWriter out = new PrintWriter(new BufferedWriter( 1597 new OutputStreamWriter(new FileOutputStream(file), Charsets.UTF_8))); 1598 metaSave(out); 1599 out.flush(); 1600 out.close(); 1601 } finally { 1602 writeUnlock(); 1603 } 1604 } 1605 1606 private void metaSave(PrintWriter out) { 1607 assert hasWriteLock(); 1608 long totalInodes = this.dir.totalInodes(); 1609 long totalBlocks = this.getBlocksTotal(); 1610 out.println(totalInodes + " files and directories, " + totalBlocks 1611 + " blocks = " + (totalInodes + totalBlocks) + " total"); 1612 1613 blockManager.metaSave(out); 1614 } 1615 1616 private String metaSaveAsString() { 1617 StringWriter sw = new StringWriter(); 1618 PrintWriter pw = new PrintWriter(sw); 1619 metaSave(pw); 1620 pw.flush(); 1621 return sw.toString(); 1622 } 1623 1624 FsServerDefaults getServerDefaults() throws StandbyException { 1625 checkOperation(OperationCategory.READ); 1626 return serverDefaults; 1627 } 1628 1629 long getAccessTimePrecision() { 1630 return accessTimePrecision; 1631 } 1632 1633 private boolean isAccessTimeSupported() { 1634 return accessTimePrecision > 0; 1635 } 1636 1637 ///////////////////////////////////////////////////////// 1638 // 1639 // These methods are called by HadoopFS clients 1640 // 1641 ///////////////////////////////////////////////////////// 1642 /** 1643 * Set permissions for an existing file. 1644 * @throws IOException 1645 */ 1646 void setPermission(String src, FsPermission permission) throws IOException { 1647 HdfsFileStatus auditStat; 1648 checkOperation(OperationCategory.WRITE); 1649 writeLock(); 1650 try { 1651 checkOperation(OperationCategory.WRITE); 1652 checkNameNodeSafeMode("Cannot set permission for " + src); 1653 auditStat = FSDirAttrOp.setPermission(dir, src, permission); 1654 } catch (AccessControlException e) { 1655 logAuditEvent(false, "setPermission", src); 1656 throw e; 1657 } finally { 1658 writeUnlock(); 1659 } 1660 getEditLog().logSync(); 1661 logAuditEvent(true, "setPermission", src, null, auditStat); 1662 } 1663 1664 /** 1665 * Set owner for an existing file. 1666 * @throws IOException 1667 */ 1668 void setOwner(String src, String username, String group) 1669 throws IOException { 1670 HdfsFileStatus auditStat; 1671 checkOperation(OperationCategory.WRITE); 1672 writeLock(); 1673 try { 1674 checkOperation(OperationCategory.WRITE); 1675 checkNameNodeSafeMode("Cannot set owner for " + src); 1676 auditStat = FSDirAttrOp.setOwner(dir, src, username, group); 1677 } catch (AccessControlException e) { 1678 logAuditEvent(false, "setOwner", src); 1679 throw e; 1680 } finally { 1681 writeUnlock(); 1682 } 1683 getEditLog().logSync(); 1684 logAuditEvent(true, "setOwner", src, null, auditStat); 1685 } 1686 1687 static class GetBlockLocationsResult { 1688 final boolean updateAccessTime; 1689 final LocatedBlocks blocks; 1690 boolean updateAccessTime() { 1691 return updateAccessTime; 1692 } 1693 private GetBlockLocationsResult( 1694 boolean updateAccessTime, LocatedBlocks blocks) { 1695 this.updateAccessTime = updateAccessTime; 1696 this.blocks = blocks; 1697 } 1698 } 1699 1700 /** 1701 * Get block locations within the specified range. 1702 * @see ClientProtocol#getBlockLocations(String, long, long) 1703 */ 1704 LocatedBlocks getBlockLocations(String clientMachine, String srcArg, 1705 long offset, long length) throws IOException { 1706 checkOperation(OperationCategory.READ); 1707 GetBlockLocationsResult res = null; 1708 FSPermissionChecker pc = getPermissionChecker(); 1709 readLock(); 1710 try { 1711 checkOperation(OperationCategory.READ); 1712 res = getBlockLocations(pc, srcArg, offset, length, true, true); 1713 } catch (AccessControlException e) { 1714 logAuditEvent(false, "open", srcArg); 1715 throw e; 1716 } finally { 1717 readUnlock(); 1718 } 1719 1720 logAuditEvent(true, "open", srcArg); 1721 1722 if (res.updateAccessTime()) { 1723 byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath( 1724 srcArg); 1725 String src = srcArg; 1726 writeLock(); 1727 final long now = now(); 1728 try { 1729 checkOperation(OperationCategory.WRITE); 1730 /** 1731 * Resolve the path again and update the atime only when the file 1732 * exists. 1733 * 1734 * XXX: Races can still occur even after resolving the path again. 1735 * For example: 1736 * 1737 * <ul> 1738 * <li>Get the block location for "/a/b"</li> 1739 * <li>Rename "/a/b" to "/c/b"</li> 1740 * <li>The second resolution still points to "/a/b", which is 1741 * wrong.</li> 1742 * </ul> 1743 * 1744 * The behavior is incorrect but consistent with the one before 1745 * HDFS-7463. A better fix is to change the edit log of SetTime to 1746 * use inode id instead of a path. 1747 */ 1748 src = dir.resolvePath(pc, srcArg, pathComponents); 1749 final INodesInPath iip = dir.getINodesInPath(src, true); 1750 INode inode = iip.getLastINode(); 1751 boolean updateAccessTime = inode != null && 1752 now > inode.getAccessTime() + getAccessTimePrecision(); 1753 if (!isInSafeMode() && updateAccessTime) { 1754 boolean changed = FSDirAttrOp.setTimes(dir, 1755 inode, -1, now, false, iip.getLatestSnapshotId()); 1756 if (changed) { 1757 getEditLog().logTimes(src, -1, now); 1758 } 1759 } 1760 } catch (Throwable e) { 1761 LOG.warn("Failed to update the access time of " + src, e); 1762 } finally { 1763 writeUnlock(); 1764 } 1765 } 1766 1767 LocatedBlocks blocks = res.blocks; 1768 if (blocks != null) { 1769 blockManager.getDatanodeManager().sortLocatedBlocks( 1770 clientMachine, blocks.getLocatedBlocks()); 1771 1772 // lastBlock is not part of getLocatedBlocks(), might need to sort it too 1773 LocatedBlock lastBlock = blocks.getLastLocatedBlock(); 1774 if (lastBlock != null) { 1775 ArrayList<LocatedBlock> lastBlockList = Lists.newArrayList(lastBlock); 1776 blockManager.getDatanodeManager().sortLocatedBlocks( 1777 clientMachine, lastBlockList); 1778 } 1779 } 1780 return blocks; 1781 } 1782 1783 /** 1784 * Get block locations within the specified range. 1785 * @see ClientProtocol#getBlockLocations(String, long, long) 1786 * @throws IOException 1787 */ 1788 GetBlockLocationsResult getBlockLocations( 1789 FSPermissionChecker pc, String src, long offset, long length, 1790 boolean needBlockToken, boolean checkSafeMode) throws IOException { 1791 if (offset < 0) { 1792 throw new HadoopIllegalArgumentException( 1793 "Negative offset is not supported. File: " + src); 1794 } 1795 if (length < 0) { 1796 throw new HadoopIllegalArgumentException( 1797 "Negative length is not supported. File: " + src); 1798 } 1799 final GetBlockLocationsResult ret = getBlockLocationsInt( 1800 pc, src, offset, length, needBlockToken); 1801 1802 if (checkSafeMode && isInSafeMode()) { 1803 for (LocatedBlock b : ret.blocks.getLocatedBlocks()) { 1804 // if safemode & no block locations yet then throw safemodeException 1805 if ((b.getLocations() == null) || (b.getLocations().length == 0)) { 1806 SafeModeException se = new SafeModeException( 1807 "Zero blocklocations for " + src, safeMode); 1808 if (haEnabled && haContext != null && 1809 haContext.getState().getServiceState() == HAServiceState.ACTIVE) { 1810 throw new RetriableException(se); 1811 } else { 1812 throw se; 1813 } 1814 } 1815 } 1816 } 1817 return ret; 1818 } 1819 1820 private GetBlockLocationsResult getBlockLocationsInt( 1821 FSPermissionChecker pc, final String srcArg, long offset, long length, 1822 boolean needBlockToken) 1823 throws IOException { 1824 String src = srcArg; 1825 byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); 1826 src = dir.resolvePath(pc, srcArg, pathComponents); 1827 final INodesInPath iip = dir.getINodesInPath(src, true); 1828 final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src); 1829 if (isPermissionEnabled) { 1830 dir.checkPathAccess(pc, iip, FsAction.READ); 1831 checkUnreadableBySuperuser(pc, inode, iip.getPathSnapshotId()); 1832 } 1833 1834 final long fileSize = iip.isSnapshot() 1835 ? inode.computeFileSize(iip.getPathSnapshotId()) 1836 : inode.computeFileSizeNotIncludingLastUcBlock(); 1837 boolean isUc = inode.isUnderConstruction(); 1838 if (iip.isSnapshot()) { 1839 // if src indicates a snapshot file, we need to make sure the returned 1840 // blocks do not exceed the size of the snapshot file. 1841 length = Math.min(length, fileSize - offset); 1842 isUc = false; 1843 } 1844 1845 final FileEncryptionInfo feInfo = 1846 FSDirectory.isReservedRawName(srcArg) ? null 1847 : dir.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip); 1848 1849 final LocatedBlocks blocks = blockManager.createLocatedBlocks( 1850 inode.getBlocks(iip.getPathSnapshotId()), fileSize, 1851 isUc, offset, length, needBlockToken, iip.isSnapshot(), feInfo); 1852 1853 // Set caching information for the located blocks. 1854 for (LocatedBlock lb : blocks.getLocatedBlocks()) { 1855 cacheManager.setCachedLocations(lb); 1856 } 1857 1858 final long now = now(); 1859 boolean updateAccessTime = isAccessTimeSupported() && !isInSafeMode() 1860 && !iip.isSnapshot() 1861 && now > inode.getAccessTime() + getAccessTimePrecision(); 1862 return new GetBlockLocationsResult(updateAccessTime, blocks); 1863 } 1864 1865 /** 1866 * Moves all the blocks from {@code srcs} and appends them to {@code target} 1867 * To avoid rollbacks we will verify validity of ALL of the args 1868 * before we start actual move. 1869 * 1870 * This does not support ".inodes" relative path 1871 * @param target target to concat into 1872 * @param srcs file that will be concatenated 1873 * @throws IOException on error 1874 */ 1875 void concat(String target, String [] srcs, boolean logRetryCache) 1876 throws IOException { 1877 waitForLoadingFSImage(); 1878 HdfsFileStatus stat = null; 1879 boolean success = false; 1880 writeLock(); 1881 try { 1882 checkOperation(OperationCategory.WRITE); 1883 checkNameNodeSafeMode("Cannot concat " + target); 1884 stat = FSDirConcatOp.concat(dir, target, srcs, logRetryCache); 1885 success = true; 1886 } finally { 1887 writeUnlock(); 1888 if (success) { 1889 getEditLog().logSync(); 1890 } 1891 logAuditEvent(success, "concat", Arrays.toString(srcs), target, stat); 1892 } 1893 } 1894 1895 /** 1896 * stores the modification and access time for this inode. 1897 * The access time is precise up to an hour. The transaction, if needed, is 1898 * written to the edits log but is not flushed. 1899 */ 1900 void setTimes(String src, long mtime, long atime) throws IOException { 1901 HdfsFileStatus auditStat; 1902 checkOperation(OperationCategory.WRITE); 1903 writeLock(); 1904 try { 1905 checkOperation(OperationCategory.WRITE); 1906 checkNameNodeSafeMode("Cannot set times " + src); 1907 auditStat = FSDirAttrOp.setTimes(dir, src, mtime, atime); 1908 } catch (AccessControlException e) { 1909 logAuditEvent(false, "setTimes", src); 1910 throw e; 1911 } finally { 1912 writeUnlock(); 1913 } 1914 getEditLog().logSync(); 1915 logAuditEvent(true, "setTimes", src, null, auditStat); 1916 } 1917 1918 /** 1919 * Create a symbolic link. 1920 */ 1921 @SuppressWarnings("deprecation") 1922 void createSymlink(String target, String link, 1923 PermissionStatus dirPerms, boolean createParent, boolean logRetryCache) 1924 throws IOException { 1925 if (!FileSystem.areSymlinksEnabled()) { 1926 throw new UnsupportedOperationException("Symlinks not supported"); 1927 } 1928 HdfsFileStatus auditStat = null; 1929 checkOperation(OperationCategory.WRITE); 1930 writeLock(); 1931 try { 1932 checkOperation(OperationCategory.WRITE); 1933 checkNameNodeSafeMode("Cannot create symlink " + link); 1934 auditStat = FSDirSymlinkOp.createSymlinkInt(this, target, link, dirPerms, 1935 createParent, logRetryCache); 1936 } catch (AccessControlException e) { 1937 logAuditEvent(false, "createSymlink", link, target, null); 1938 throw e; 1939 } finally { 1940 writeUnlock(); 1941 } 1942 getEditLog().logSync(); 1943 logAuditEvent(true, "createSymlink", link, target, auditStat); 1944 } 1945 1946 /** 1947 * Set replication for an existing file. 1948 * 1949 * The NameNode sets new replication and schedules either replication of 1950 * under-replicated data blocks or removal of the excessive block copies 1951 * if the blocks are over-replicated. 1952 * 1953 * @see ClientProtocol#setReplication(String, short) 1954 * @param src file name 1955 * @param replication new replication 1956 * @return true if successful; 1957 * false if file does not exist or is a directory 1958 */ 1959 boolean setReplication(final String src, final short replication) 1960 throws IOException { 1961 boolean success = false; 1962 waitForLoadingFSImage(); 1963 checkOperation(OperationCategory.WRITE); 1964 writeLock(); 1965 try { 1966 checkOperation(OperationCategory.WRITE); 1967 checkNameNodeSafeMode("Cannot set replication for " + src); 1968 success = FSDirAttrOp.setReplication(dir, blockManager, src, replication); 1969 } catch (AccessControlException e) { 1970 logAuditEvent(false, "setReplication", src); 1971 throw e; 1972 } finally { 1973 writeUnlock(); 1974 } 1975 if (success) { 1976 getEditLog().logSync(); 1977 logAuditEvent(true, "setReplication", src); 1978 } 1979 return success; 1980 } 1981 1982 /** 1983 * Truncate file to a lower length. 1984 * Truncate cannot be reverted / recovered from as it causes data loss. 1985 * Truncation at block boundary is atomic, otherwise it requires 1986 * block recovery to truncate the last block of the file. 1987 * 1988 * @return true if client does not need to wait for block recovery, 1989 * false if client needs to wait for block recovery. 1990 */ 1991 boolean truncate(String src, long newLength, 1992 String clientName, String clientMachine, 1993 long mtime) 1994 throws IOException, UnresolvedLinkException { 1995 boolean ret; 1996 try { 1997 ret = truncateInt(src, newLength, clientName, clientMachine, mtime); 1998 } catch (AccessControlException e) { 1999 logAuditEvent(false, "truncate", src); 2000 throw e; 2001 } 2002 return ret; 2003 } 2004 2005 boolean truncateInt(String srcArg, long newLength, 2006 String clientName, String clientMachine, 2007 long mtime) 2008 throws IOException, UnresolvedLinkException { 2009 String src = srcArg; 2010 NameNode.stateChangeLog.debug( 2011 "DIR* NameSystem.truncate: src={} newLength={}", src, newLength); 2012 if (newLength < 0) { 2013 throw new HadoopIllegalArgumentException( 2014 "Cannot truncate to a negative file size: " + newLength + "."); 2015 } 2016 HdfsFileStatus stat = null; 2017 FSPermissionChecker pc = getPermissionChecker(); 2018 checkOperation(OperationCategory.WRITE); 2019 boolean res; 2020 byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); 2021 writeLock(); 2022 BlocksMapUpdateInfo toRemoveBlocks = new BlocksMapUpdateInfo(); 2023 try { 2024 checkOperation(OperationCategory.WRITE); 2025 checkNameNodeSafeMode("Cannot truncate for " + src); 2026 src = dir.resolvePath(pc, src, pathComponents); 2027 res = truncateInternal(src, newLength, clientName, 2028 clientMachine, mtime, pc, toRemoveBlocks); 2029 stat = dir.getAuditFileInfo(dir.getINodesInPath4Write(src, false)); 2030 } finally { 2031 writeUnlock(); 2032 } 2033 getEditLog().logSync(); 2034 if (!toRemoveBlocks.getToDeleteList().isEmpty()) { 2035 removeBlocks(toRemoveBlocks); 2036 toRemoveBlocks.clear(); 2037 } 2038 logAuditEvent(true, "truncate", src, null, stat); 2039 return res; 2040 } 2041 2042 /** 2043 * Truncate a file to a given size 2044 * Update the count at each ancestor directory with quota 2045 */ 2046 boolean truncateInternal(String src, long newLength, 2047 String clientName, String clientMachine, 2048 long mtime, FSPermissionChecker pc, 2049 BlocksMapUpdateInfo toRemoveBlocks) 2050 throws IOException, UnresolvedLinkException { 2051 assert hasWriteLock(); 2052 INodesInPath iip = dir.getINodesInPath4Write(src, true); 2053 if (isPermissionEnabled) { 2054 dir.checkPathAccess(pc, iip, FsAction.WRITE); 2055 } 2056 INodeFile file = INodeFile.valueOf(iip.getLastINode(), src); 2057 final BlockStoragePolicy lpPolicy = 2058 blockManager.getStoragePolicy("LAZY_PERSIST"); 2059 2060 if (lpPolicy != null && 2061 lpPolicy.getId() == file.getStoragePolicyID()) { 2062 throw new UnsupportedOperationException( 2063 "Cannot truncate lazy persist file " + src); 2064 } 2065 2066 // Check if the file is already being truncated with the same length 2067 final BlockInfoContiguous last = file.getLastBlock(); 2068 if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) { 2069 final Block truncateBlock 2070 = ((BlockInfoContiguousUnderConstruction)last).getTruncateBlock(); 2071 if (truncateBlock != null) { 2072 final long truncateLength = file.computeFileSize(false, false) 2073 + truncateBlock.getNumBytes(); 2074 if (newLength == truncateLength) { 2075 return false; 2076 } 2077 } 2078 } 2079 2080 // Opening an existing file for truncate. May need lease recovery. 2081 recoverLeaseInternal(RecoverLeaseOp.TRUNCATE_FILE, 2082 iip, src, clientName, clientMachine, false); 2083 // Truncate length check. 2084 long oldLength = file.computeFileSize(); 2085 if(oldLength == newLength) { 2086 return true; 2087 } 2088 if(oldLength < newLength) { 2089 throw new HadoopIllegalArgumentException( 2090 "Cannot truncate to a larger file size. Current size: " + oldLength + 2091 ", truncate size: " + newLength + "."); 2092 } 2093 // Perform INodeFile truncation. 2094 final QuotaCounts delta = new QuotaCounts.Builder().build(); 2095 boolean onBlockBoundary = dir.truncate(iip, newLength, toRemoveBlocks, 2096 mtime, delta); 2097 Block truncateBlock = null; 2098 if(!onBlockBoundary) { 2099 // Open file for write, but don't log into edits 2100 long lastBlockDelta = file.computeFileSize() - newLength; 2101 assert lastBlockDelta > 0 : "delta is 0 only if on block bounday"; 2102 truncateBlock = prepareFileForTruncate(iip, clientName, clientMachine, 2103 lastBlockDelta, null); 2104 } 2105 2106 // update the quota: use the preferred block size for UC block 2107 dir.writeLock(); 2108 try { 2109 dir.updateCountNoQuotaCheck(iip, iip.length() - 1, delta); 2110 } finally { 2111 dir.writeUnlock(); 2112 } 2113 2114 getEditLog().logTruncate(src, clientName, clientMachine, newLength, mtime, 2115 truncateBlock); 2116 return onBlockBoundary; 2117 } 2118 2119 /** 2120 * Convert current INode to UnderConstruction. 2121 * Recreate lease. 2122 * Create new block for the truncated copy. 2123 * Schedule truncation of the replicas. 2124 * 2125 * @return the returned block will be written to editLog and passed back into 2126 * this method upon loading. 2127 */ 2128 Block prepareFileForTruncate(INodesInPath iip, 2129 String leaseHolder, 2130 String clientMachine, 2131 long lastBlockDelta, 2132 Block newBlock) 2133 throws IOException { 2134 INodeFile file = iip.getLastINode().asFile(); 2135 String src = iip.getPath(); 2136 file.recordModification(iip.getLatestSnapshotId()); 2137 file.toUnderConstruction(leaseHolder, clientMachine); 2138 assert file.isUnderConstruction() : "inode should be under construction."; 2139 leaseManager.addLease( 2140 file.getFileUnderConstructionFeature().getClientName(), src); 2141 boolean shouldRecoverNow = (newBlock == null); 2142 BlockInfoContiguous oldBlock = file.getLastBlock(); 2143 boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, oldBlock); 2144 if(newBlock == null) { 2145 newBlock = (shouldCopyOnTruncate) ? createNewBlock() : 2146 new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(), 2147 nextGenerationStamp(blockIdManager.isLegacyBlock(oldBlock))); 2148 } 2149 2150 BlockInfoContiguousUnderConstruction truncatedBlockUC; 2151 if(shouldCopyOnTruncate) { 2152 // Add new truncateBlock into blocksMap and 2153 // use oldBlock as a source for copy-on-truncate recovery 2154 truncatedBlockUC = new BlockInfoContiguousUnderConstruction(newBlock, 2155 file.getBlockReplication()); 2156 truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta); 2157 truncatedBlockUC.setTruncateBlock(oldBlock); 2158 file.setLastBlock(truncatedBlockUC, blockManager.getStorages(oldBlock)); 2159 getBlockManager().addBlockCollection(truncatedBlockUC, file); 2160 2161 NameNode.stateChangeLog.debug( 2162 "BLOCK* prepareFileForTruncate: Scheduling copy-on-truncate to new" + 2163 " size {} new block {} old block {}", truncatedBlockUC.getNumBytes(), 2164 newBlock, truncatedBlockUC.getTruncateBlock()); 2165 } else { 2166 // Use new generation stamp for in-place truncate recovery 2167 blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta); 2168 oldBlock = file.getLastBlock(); 2169 assert !oldBlock.isComplete() : "oldBlock should be under construction"; 2170 truncatedBlockUC = (BlockInfoContiguousUnderConstruction) oldBlock; 2171 truncatedBlockUC.setTruncateBlock(new Block(oldBlock)); 2172 truncatedBlockUC.getTruncateBlock().setNumBytes( 2173 oldBlock.getNumBytes() - lastBlockDelta); 2174 truncatedBlockUC.getTruncateBlock().setGenerationStamp( 2175 newBlock.getGenerationStamp()); 2176 2177 NameNode.stateChangeLog.debug( 2178 "BLOCK* prepareFileForTruncate: {} Scheduling in-place block " + 2179 "truncate to new size {}", 2180 truncatedBlockUC.getTruncateBlock().getNumBytes(), truncatedBlockUC); 2181 } 2182 if (shouldRecoverNow) { 2183 truncatedBlockUC.initializeBlockRecovery(newBlock.getGenerationStamp()); 2184 } 2185 2186 return newBlock; 2187 } 2188 2189 /** 2190 * Defines if a replica needs to be copied on truncate or 2191 * can be truncated in place. 2192 */ 2193 boolean shouldCopyOnTruncate(INodeFile file, BlockInfoContiguous blk) { 2194 if(!isUpgradeFinalized()) { 2195 return true; 2196 } 2197 if (isRollingUpgrade()) { 2198 return true; 2199 } 2200 return file.isBlockInLatestSnapshot(blk); 2201 } 2202 2203 /** 2204 * Set the storage policy for a file or a directory. 2205 * 2206 * @param src file/directory path 2207 * @param policyName storage policy name 2208 */ 2209 void setStoragePolicy(String src, String policyName) throws IOException { 2210 HdfsFileStatus auditStat; 2211 waitForLoadingFSImage(); 2212 checkOperation(OperationCategory.WRITE); 2213 writeLock(); 2214 try { 2215 checkOperation(OperationCategory.WRITE); 2216 checkNameNodeSafeMode("Cannot set storage policy for " + src); 2217 auditStat = FSDirAttrOp.setStoragePolicy( 2218 dir, blockManager, src, policyName); 2219 } catch (AccessControlException e) { 2220 logAuditEvent(false, "setStoragePolicy", src); 2221 throw e; 2222 } finally { 2223 writeUnlock(); 2224 } 2225 getEditLog().logSync(); 2226 logAuditEvent(true, "setStoragePolicy", src, null, auditStat); 2227 } 2228 2229 /** 2230 * @return All the existing block storage policies 2231 */ 2232 BlockStoragePolicy[] getStoragePolicies() throws IOException { 2233 checkOperation(OperationCategory.READ); 2234 waitForLoadingFSImage(); 2235 readLock(); 2236 try { 2237 checkOperation(OperationCategory.READ); 2238 return FSDirAttrOp.getStoragePolicies(blockManager); 2239 } finally { 2240 readUnlock(); 2241 } 2242 } 2243 2244 long getPreferredBlockSize(String src) throws IOException { 2245 checkOperation(OperationCategory.READ); 2246 readLock(); 2247 try { 2248 checkOperation(OperationCategory.READ); 2249 return FSDirAttrOp.getPreferredBlockSize(dir, src); 2250 } finally { 2251 readUnlock(); 2252 } 2253 } 2254 2255 /** 2256 * If the file is within an encryption zone, select the appropriate 2257 * CryptoProtocolVersion from the list provided by the client. Since the 2258 * client may be newer, we need to handle unknown versions. 2259 * 2260 * @param zone EncryptionZone of the file 2261 * @param supportedVersions List of supported protocol versions 2262 * @return chosen protocol version 2263 * @throws IOException 2264 */ 2265 private CryptoProtocolVersion chooseProtocolVersion(EncryptionZone zone, 2266 CryptoProtocolVersion[] supportedVersions) 2267 throws UnknownCryptoProtocolVersionException, UnresolvedLinkException, 2268 SnapshotAccessControlException { 2269 Preconditions.checkNotNull(zone); 2270 Preconditions.checkNotNull(supportedVersions); 2271 // Right now, we only support a single protocol version, 2272 // so simply look for it in the list of provided options 2273 final CryptoProtocolVersion required = zone.getVersion(); 2274 2275 for (CryptoProtocolVersion c : supportedVersions) { 2276 if (c.equals(CryptoProtocolVersion.UNKNOWN)) { 2277 if (LOG.isDebugEnabled()) { 2278 LOG.debug("Ignoring unknown CryptoProtocolVersion provided by " + 2279 "client: " + c.getUnknownValue()); 2280 } 2281 continue; 2282 } 2283 if (c.equals(required)) { 2284 return c; 2285 } 2286 } 2287 throw new UnknownCryptoProtocolVersionException( 2288 "No crypto protocol versions provided by the client are supported." 2289 + " Client provided: " + Arrays.toString(supportedVersions) 2290 + " NameNode supports: " + Arrays.toString(CryptoProtocolVersion 2291 .values())); 2292 } 2293 2294 /** 2295 * Invoke KeyProvider APIs to generate an encrypted data encryption key for an 2296 * encryption zone. Should not be called with any locks held. 2297 * 2298 * @param ezKeyName key name of an encryption zone 2299 * @return New EDEK, or null if ezKeyName is null 2300 * @throws IOException 2301 */ 2302 private EncryptedKeyVersion generateEncryptedDataEncryptionKey(String 2303 ezKeyName) throws IOException { 2304 if (ezKeyName == null) { 2305 return null; 2306 } 2307 EncryptedKeyVersion edek = null; 2308 try { 2309 edek = provider.generateEncryptedKey(ezKeyName); 2310 } catch (GeneralSecurityException e) { 2311 throw new IOException(e); 2312 } 2313 Preconditions.checkNotNull(edek); 2314 return edek; 2315 } 2316 2317 /** 2318 * Create a new file entry in the namespace. 2319 * 2320 * For description of parameters and exceptions thrown see 2321 * {@link ClientProtocol#create}, except it returns valid file status upon 2322 * success 2323 */ 2324 HdfsFileStatus startFile(String src, PermissionStatus permissions, 2325 String holder, String clientMachine, EnumSet<CreateFlag> flag, 2326 boolean createParent, short replication, long blockSize, 2327 CryptoProtocolVersion[] supportedVersions, boolean logRetryCache) 2328 throws AccessControlException, SafeModeException, 2329 FileAlreadyExistsException, UnresolvedLinkException, 2330 FileNotFoundException, ParentNotDirectoryException, IOException { 2331 2332 HdfsFileStatus status = null; 2333 try { 2334 status = startFileInt(src, permissions, holder, clientMachine, flag, 2335 createParent, replication, blockSize, supportedVersions, 2336 logRetryCache); 2337 } catch (AccessControlException e) { 2338 logAuditEvent(false, "create", src); 2339 throw e; 2340 } 2341 return status; 2342 } 2343 2344 private HdfsFileStatus startFileInt(final String srcArg, 2345 PermissionStatus permissions, String holder, String clientMachine, 2346 EnumSet<CreateFlag> flag, boolean createParent, short replication, 2347 long blockSize, CryptoProtocolVersion[] supportedVersions, 2348 boolean logRetryCache) 2349 throws AccessControlException, SafeModeException, 2350 FileAlreadyExistsException, UnresolvedLinkException, 2351 FileNotFoundException, ParentNotDirectoryException, IOException { 2352 String src = srcArg; 2353 if (NameNode.stateChangeLog.isDebugEnabled()) { 2354 StringBuilder builder = new StringBuilder(); 2355 builder.append("DIR* NameSystem.startFile: src=" + src 2356 + ", holder=" + holder 2357 + ", clientMachine=" + clientMachine 2358 + ", createParent=" + createParent 2359 + ", replication=" + replication 2360 + ", createFlag=" + flag.toString() 2361 + ", blockSize=" + blockSize); 2362 builder.append(", supportedVersions="); 2363 if (supportedVersions != null) { 2364 builder.append(Arrays.toString(supportedVersions)); 2365 } else { 2366 builder.append("null"); 2367 } 2368 NameNode.stateChangeLog.debug(builder.toString()); 2369 } 2370 if (!DFSUtil.isValidName(src)) { 2371 throw new InvalidPathException(src); 2372 } 2373 blockManager.verifyReplication(src, replication, clientMachine); 2374 2375 boolean skipSync = false; 2376 HdfsFileStatus stat = null; 2377 FSPermissionChecker pc = getPermissionChecker(); 2378 if (blockSize < minBlockSize) { 2379 throw new IOException("Specified block size is less than configured" + 2380 " minimum value (" + DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY 2381 + "): " + blockSize + " < " + minBlockSize); 2382 } 2383 byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); 2384 boolean create = flag.contains(CreateFlag.CREATE); 2385 boolean overwrite = flag.contains(CreateFlag.OVERWRITE); 2386 boolean isLazyPersist = flag.contains(CreateFlag.LAZY_PERSIST); 2387 2388 waitForLoadingFSImage(); 2389 2390 /** 2391 * If the file is in an encryption zone, we optimistically create an 2392 * EDEK for the file by calling out to the configured KeyProvider. 2393 * Since this typically involves doing an RPC, we take the readLock 2394 * initially, then drop it to do the RPC. 2395 * 2396 * Since the path can flip-flop between being in an encryption zone and not 2397 * in the meantime, we need to recheck the preconditions when we retake the 2398 * lock to do the create. If the preconditions are not met, we throw a 2399 * special RetryStartFileException to ask the DFSClient to try the create 2400 * again later. 2401 */ 2402 CryptoProtocolVersion protocolVersion = null; 2403 CipherSuite suite = null; 2404 String ezKeyName = null; 2405 EncryptedKeyVersion edek = null; 2406 2407 if (provider != null) { 2408 readLock(); 2409 try { 2410 src = dir.resolvePath(pc, src, pathComponents); 2411 INodesInPath iip = dir.getINodesInPath4Write(src); 2412 // Nothing to do if the path is not within an EZ 2413 final EncryptionZone zone = dir.getEZForPath(iip); 2414 if (zone != null) { 2415 protocolVersion = chooseProtocolVersion(zone, supportedVersions); 2416 suite = zone.getSuite(); 2417 ezKeyName = zone.getKeyName(); 2418 2419 Preconditions.checkNotNull(protocolVersion); 2420 Preconditions.checkNotNull(suite); 2421 Preconditions.checkArgument(!suite.equals(CipherSuite.UNKNOWN), 2422 "Chose an UNKNOWN CipherSuite!"); 2423 Preconditions.checkNotNull(ezKeyName); 2424 } 2425 } finally { 2426 readUnlock(); 2427 } 2428 2429 Preconditions.checkState( 2430 (suite == null && ezKeyName == null) || 2431 (suite != null && ezKeyName != null), 2432 "Both suite and ezKeyName should both be null or not null"); 2433 2434 // Generate EDEK if necessary while not holding the lock 2435 edek = generateEncryptedDataEncryptionKey(ezKeyName); 2436 EncryptionFaultInjector.getInstance().startFileAfterGenerateKey(); 2437 } 2438 2439 // Proceed with the create, using the computed cipher suite and 2440 // generated EDEK 2441 BlocksMapUpdateInfo toRemoveBlocks = null; 2442 writeLock(); 2443 try { 2444 checkOperation(OperationCategory.WRITE); 2445 checkNameNodeSafeMode("Cannot create file" + src); 2446 dir.writeLock(); 2447 try { 2448 src = dir.resolvePath(pc, src, pathComponents); 2449 final INodesInPath iip = dir.getINodesInPath4Write(src); 2450 toRemoveBlocks = startFileInternal( 2451 pc, iip, permissions, holder, 2452 clientMachine, create, overwrite, 2453 createParent, replication, blockSize, 2454 isLazyPersist, suite, protocolVersion, edek, 2455 logRetryCache); 2456 stat = FSDirStatAndListingOp.getFileInfo( 2457 dir, src, false, FSDirectory.isReservedRawName(srcArg), true); 2458 } finally { 2459 dir.writeUnlock(); 2460 } 2461 } catch (StandbyException se) { 2462 skipSync = true; 2463 throw se; 2464 } finally { 2465 writeUnlock(); 2466 // There might be transactions logged while trying to recover the lease. 2467 // They need to be sync'ed even when an exception was thrown. 2468 if (!skipSync) { 2469 getEditLog().logSync(); 2470 if (toRemoveBlocks != null) { 2471 removeBlocks(toRemoveBlocks); 2472 toRemoveBlocks.clear(); 2473 } 2474 } 2475 } 2476 2477 logAuditEvent(true, "create", srcArg, null, stat); 2478 return stat; 2479 } 2480 2481 /** 2482 * Create a new file or overwrite an existing file<br> 2483 * 2484 * Once the file is create the client then allocates a new block with the next 2485 * call using {@link ClientProtocol#addBlock}. 2486 * <p> 2487 * For description of parameters and exceptions thrown see 2488 * {@link ClientProtocol#create} 2489 */ 2490 private BlocksMapUpdateInfo startFileInternal(FSPermissionChecker pc, 2491 INodesInPath iip, PermissionStatus permissions, String holder, 2492 String clientMachine, boolean create, boolean overwrite, 2493 boolean createParent, short replication, long blockSize, 2494 boolean isLazyPersist, CipherSuite suite, CryptoProtocolVersion version, 2495 EncryptedKeyVersion edek, boolean logRetryEntry) 2496 throws IOException { 2497 assert hasWriteLock(); 2498 // Verify that the destination does not exist as a directory already. 2499 final INode inode = iip.getLastINode(); 2500 final String src = iip.getPath(); 2501 if (inode != null && inode.isDirectory()) { 2502 throw new FileAlreadyExistsException(src + 2503 " already exists as a directory"); 2504 } 2505 2506 final INodeFile myFile = INodeFile.valueOf(inode, src, true); 2507 if (isPermissionEnabled) { 2508 if (overwrite && myFile != null) { 2509 dir.checkPathAccess(pc, iip, FsAction.WRITE); 2510 } 2511 /* 2512 * To overwrite existing file, need to check 'w' permission 2513 * of parent (equals to ancestor in this case) 2514 */ 2515 dir.checkAncestorAccess(pc, iip, FsAction.WRITE); 2516 } 2517 if (!createParent) { 2518 dir.verifyParentDir(iip, src); 2519 } 2520 2521 FileEncryptionInfo feInfo = null; 2522 2523 final EncryptionZone zone = dir.getEZForPath(iip); 2524 if (zone != null) { 2525 // The path is now within an EZ, but we're missing encryption parameters 2526 if (suite == null || edek == null) { 2527 throw new RetryStartFileException(); 2528 } 2529 // Path is within an EZ and we have provided encryption parameters. 2530 // Make sure that the generated EDEK matches the settings of the EZ. 2531 final String ezKeyName = zone.getKeyName(); 2532 if (!ezKeyName.equals(edek.getEncryptionKeyName())) { 2533 throw new RetryStartFileException(); 2534 } 2535 feInfo = new FileEncryptionInfo(suite, version, 2536 edek.getEncryptedKeyVersion().getMaterial(), 2537 edek.getEncryptedKeyIv(), 2538 ezKeyName, edek.getEncryptionKeyVersionName()); 2539 } 2540 2541 try { 2542 BlocksMapUpdateInfo toRemoveBlocks = null; 2543 if (myFile == null) { 2544 if (!create) { 2545 throw new FileNotFoundException("Can't overwrite non-existent " + 2546 src + " for client " + clientMachine); 2547 } 2548 } else { 2549 if (overwrite) { 2550 toRemoveBlocks = new BlocksMapUpdateInfo(); 2551 List<INode> toRemoveINodes = new ChunkedArrayList<INode>(); 2552 long ret = FSDirDeleteOp.delete(dir, iip, toRemoveBlocks, 2553 toRemoveINodes, now()); 2554 if (ret >= 0) { 2555 iip = INodesInPath.replace(iip, iip.length() - 1, null); 2556 FSDirDeleteOp.incrDeletedFileCount(ret); 2557 removeLeasesAndINodes(src, toRemoveINodes, true); 2558 } 2559 } else { 2560 // If lease soft limit time is expired, recover the lease 2561 recoverLeaseInternal(RecoverLeaseOp.CREATE_FILE, 2562 iip, src, holder, clientMachine, false); 2563 throw new FileAlreadyExistsException(src + " for client " + 2564 clientMachine + " already exists"); 2565 } 2566 } 2567 2568 checkFsObjectLimit(); 2569 INodeFile newNode = null; 2570 2571 // Always do an implicit mkdirs for parent directory tree. 2572 Map.Entry<INodesInPath, String> parent = FSDirMkdirOp 2573 .createAncestorDirectories(dir, iip, permissions); 2574 if (parent != null) { 2575 iip = dir.addFile(parent.getKey(), parent.getValue(), permissions, 2576 replication, blockSize, holder, clientMachine); 2577 newNode = iip != null ? iip.getLastINode().asFile() : null; 2578 } 2579 2580 if (newNode == null) { 2581 throw new IOException("Unable to add " + src + " to namespace"); 2582 } 2583 leaseManager.addLease(newNode.getFileUnderConstructionFeature() 2584 .getClientName(), src); 2585 2586 // Set encryption attributes if necessary 2587 if (feInfo != null) { 2588 dir.setFileEncryptionInfo(src, feInfo); 2589 newNode = dir.getInode(newNode.getId()).asFile(); 2590 } 2591 2592 setNewINodeStoragePolicy(newNode, iip, isLazyPersist); 2593 2594 // record file record in log, record new generation stamp 2595 getEditLog().logOpenFile(src, newNode, overwrite, logRetryEntry); 2596 NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: added {}" + 2597 " inode {} holder {}", src, newNode.getId(), holder); 2598 return toRemoveBlocks; 2599 } catch (IOException ie) { 2600 NameNode.stateChangeLog.warn("DIR* NameSystem.startFile: " + src + " " + 2601 ie.getMessage()); 2602 throw ie; 2603 } 2604 } 2605 2606 private void setNewINodeStoragePolicy(INodeFile inode, 2607 INodesInPath iip, 2608 boolean isLazyPersist) 2609 throws IOException { 2610 2611 if (isLazyPersist) { 2612 BlockStoragePolicy lpPolicy = 2613 blockManager.getStoragePolicy("LAZY_PERSIST"); 2614 2615 // Set LAZY_PERSIST storage policy if the flag was passed to 2616 // CreateFile. 2617 if (lpPolicy == null) { 2618 throw new HadoopIllegalArgumentException( 2619 "The LAZY_PERSIST storage policy has been disabled " + 2620 "by the administrator."); 2621 } 2622 inode.setStoragePolicyID(lpPolicy.getId(), 2623 iip.getLatestSnapshotId()); 2624 } else { 2625 BlockStoragePolicy effectivePolicy = 2626 blockManager.getStoragePolicy(inode.getStoragePolicyID()); 2627 2628 if (effectivePolicy != null && 2629 effectivePolicy.isCopyOnCreateFile()) { 2630 // Copy effective policy from ancestor directory to current file. 2631 inode.setStoragePolicyID(effectivePolicy.getId(), 2632 iip.getLatestSnapshotId()); 2633 } 2634 } 2635 } 2636 2637 /** 2638 * Append to an existing file for append. 2639 * <p> 2640 * 2641 * The method returns the last block of the file if this is a partial block, 2642 * which can still be used for writing more data. The client uses the returned 2643 * block locations to form the data pipeline for this block.<br> 2644 * The method returns null if the last block is full. The client then 2645 * allocates a new block with the next call using 2646 * {@link ClientProtocol#addBlock}. 2647 * <p> 2648 * 2649 * For description of parameters and exceptions thrown see 2650 * {@link ClientProtocol#append(String, String, EnumSetWritable)} 2651 * 2652 * @return the last block locations if the block is partial or null otherwise 2653 */ 2654 private LocatedBlock appendFileInternal(FSPermissionChecker pc, 2655 INodesInPath iip, String holder, String clientMachine, boolean newBlock, 2656 boolean logRetryCache) throws IOException { 2657 assert hasWriteLock(); 2658 // Verify that the destination does not exist as a directory already. 2659 final INode inode = iip.getLastINode(); 2660 final String src = iip.getPath(); 2661 if (inode != null && inode.isDirectory()) { 2662 throw new FileAlreadyExistsException("Cannot append to directory " + src 2663 + "; already exists as a directory."); 2664 } 2665 if (isPermissionEnabled) { 2666 dir.checkPathAccess(pc, iip, FsAction.WRITE); 2667 } 2668 2669 try { 2670 if (inode == null) { 2671 throw new FileNotFoundException("failed to append to non-existent file " 2672 + src + " for client " + clientMachine); 2673 } 2674 INodeFile myFile = INodeFile.valueOf(inode, src, true); 2675 final BlockStoragePolicy lpPolicy = 2676 blockManager.getStoragePolicy("LAZY_PERSIST"); 2677 if (lpPolicy != null && 2678 lpPolicy.getId() == myFile.getStoragePolicyID()) { 2679 throw new UnsupportedOperationException( 2680 "Cannot append to lazy persist file " + src); 2681 } 2682 // Opening an existing file for append - may need to recover lease. 2683 recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE, 2684 iip, src, holder, clientMachine, false); 2685 2686 final BlockInfoContiguous lastBlock = myFile.getLastBlock(); 2687 // Check that the block has at least minimum replication. 2688 if(lastBlock != null && lastBlock.isComplete() && 2689 !getBlockManager().isSufficientlyReplicated(lastBlock)) { 2690 throw new IOException("append: lastBlock=" + lastBlock + 2691 " of src=" + src + " is not sufficiently replicated yet."); 2692 } 2693 return prepareFileForAppend(src, iip, holder, clientMachine, newBlock, 2694 true, logRetryCache); 2695 } catch (IOException ie) { 2696 NameNode.stateChangeLog.warn("DIR* NameSystem.append: " +ie.getMessage()); 2697 throw ie; 2698 } 2699 } 2700 2701 /** 2702 * Convert current node to under construction. 2703 * Recreate in-memory lease record. 2704 * 2705 * @param src path to the file 2706 * @param leaseHolder identifier of the lease holder on this file 2707 * @param clientMachine identifier of the client machine 2708 * @param newBlock if the data is appended to a new block 2709 * @param writeToEditLog whether to persist this change to the edit log 2710 * @param logRetryCache whether to record RPC ids in editlog for retry cache 2711 * rebuilding 2712 * @return the last block locations if the block is partial or null otherwise 2713 * @throws UnresolvedLinkException 2714 * @throws IOException 2715 */ 2716 LocatedBlock prepareFileForAppend(String src, INodesInPath iip, 2717 String leaseHolder, String clientMachine, boolean newBlock, 2718 boolean writeToEditLog, boolean logRetryCache) throws IOException { 2719 final INodeFile file = iip.getLastINode().asFile(); 2720 final QuotaCounts delta = verifyQuotaForUCBlock(file, iip); 2721 2722 file.recordModification(iip.getLatestSnapshotId()); 2723 file.toUnderConstruction(leaseHolder, clientMachine); 2724 2725 leaseManager.addLease( 2726 file.getFileUnderConstructionFeature().getClientName(), src); 2727 2728 LocatedBlock ret = null; 2729 if (!newBlock) { 2730 ret = blockManager.convertLastBlockToUnderConstruction(file, 0); 2731 if (ret != null && delta != null) { 2732 Preconditions.checkState(delta.getStorageSpace() >= 0, 2733 "appending to a block with size larger than the preferred block size"); 2734 dir.writeLock(); 2735 try { 2736 dir.updateCountNoQuotaCheck(iip, iip.length() - 1, delta); 2737 } finally { 2738 dir.writeUnlock(); 2739 } 2740 } 2741 } else { 2742 BlockInfoContiguous lastBlock = file.getLastBlock(); 2743 if (lastBlock != null) { 2744 ExtendedBlock blk = new ExtendedBlock(this.getBlockPoolId(), lastBlock); 2745 ret = new LocatedBlock(blk, new DatanodeInfo[0]); 2746 } 2747 } 2748 2749 if (writeToEditLog) { 2750 getEditLog().logAppendFile(src, file, newBlock, logRetryCache); 2751 } 2752 return ret; 2753 } 2754 2755 /** 2756 * Verify quota when using the preferred block size for UC block. This is 2757 * usually used by append and truncate 2758 * @throws QuotaExceededException when violating the storage quota 2759 * @return expected quota usage update. null means no change or no need to 2760 * update quota usage later 2761 */ 2762 private QuotaCounts verifyQuotaForUCBlock(INodeFile file, INodesInPath iip) 2763 throws QuotaExceededException { 2764 if (!isImageLoaded() || dir.shouldSkipQuotaChecks()) { 2765 // Do not check quota if editlog is still being processed 2766 return null; 2767 } 2768 if (file.getLastBlock() != null) { 2769 final QuotaCounts delta = computeQuotaDeltaForUCBlock(file); 2770 dir.readLock(); 2771 try { 2772 FSDirectory.verifyQuota(iip, iip.length() - 1, delta, null); 2773 return delta; 2774 } finally { 2775 dir.readUnlock(); 2776 } 2777 } 2778 return null; 2779 } 2780 2781 /** Compute quota change for converting a complete block to a UC block */ 2782 private QuotaCounts computeQuotaDeltaForUCBlock(INodeFile file) { 2783 final QuotaCounts delta = new QuotaCounts.Builder().build(); 2784 final BlockInfoContiguous lastBlock = file.getLastBlock(); 2785 if (lastBlock != null) { 2786 final long diff = file.getPreferredBlockSize() - lastBlock.getNumBytes(); 2787 final short repl = file.getBlockReplication(); 2788 delta.addStorageSpace(diff * repl); 2789 final BlockStoragePolicy policy = dir.getBlockStoragePolicySuite() 2790 .getPolicy(file.getStoragePolicyID()); 2791 List<StorageType> types = policy.chooseStorageTypes(repl); 2792 for (StorageType t : types) { 2793 if (t.supportTypeQuota()) { 2794 delta.addTypeSpace(t, diff); 2795 } 2796 } 2797 } 2798 return delta; 2799 } 2800 2801 /** 2802 * Recover lease; 2803 * Immediately revoke the lease of the current lease holder and start lease 2804 * recovery so that the file can be forced to be closed. 2805 * 2806 * @param src the path of the file to start lease recovery 2807 * @param holder the lease holder's name 2808 * @param clientMachine the client machine's name 2809 * @return true if the file is already closed or 2810 * if the lease can be released and the file can be closed. 2811 * @throws IOException 2812 */ 2813 boolean recoverLease(String src, String holder, String clientMachine) 2814 throws IOException { 2815 if (!DFSUtil.isValidName(src)) { 2816 throw new IOException("Invalid file name: " + src); 2817 } 2818 2819 boolean skipSync = false; 2820 FSPermissionChecker pc = getPermissionChecker(); 2821 checkOperation(OperationCategory.WRITE); 2822 byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); 2823 writeLock(); 2824 try { 2825 checkOperation(OperationCategory.WRITE); 2826 checkNameNodeSafeMode("Cannot recover the lease of " + src); 2827 src = dir.resolvePath(pc, src, pathComponents); 2828 final INodesInPath iip = dir.getINodesInPath4Write(src); 2829 final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src); 2830 if (!inode.isUnderConstruction()) { 2831 return true; 2832 } 2833 if (isPermissionEnabled) { 2834 dir.checkPathAccess(pc, iip, FsAction.WRITE); 2835 } 2836 2837 return recoverLeaseInternal(RecoverLeaseOp.RECOVER_LEASE, 2838 iip, src, holder, clientMachine, true); 2839 } catch (StandbyException se) { 2840 skipSync = true; 2841 throw se; 2842 } finally { 2843 writeUnlock(); 2844 // There might be transactions logged while trying to recover the lease. 2845 // They need to be sync'ed even when an exception was thrown. 2846 if (!skipSync) { 2847 getEditLog().logSync(); 2848 } 2849 } 2850 } 2851 2852 private enum RecoverLeaseOp { 2853 CREATE_FILE, 2854 APPEND_FILE, 2855 TRUNCATE_FILE, 2856 RECOVER_LEASE; 2857 2858 private String getExceptionMessage(String src, String holder, 2859 String clientMachine, String reason) { 2860 return "Failed to " + this + " " + src + " for " + holder + 2861 " on " + clientMachine + " because " + reason; 2862 } 2863 } 2864 2865 boolean recoverLeaseInternal(RecoverLeaseOp op, INodesInPath iip, 2866 String src, String holder, String clientMachine, boolean force) 2867 throws IOException { 2868 assert hasWriteLock(); 2869 INodeFile file = iip.getLastINode().asFile(); 2870 if (file.isUnderConstruction()) { 2871 // 2872 // If the file is under construction , then it must be in our 2873 // leases. Find the appropriate lease record. 2874 // 2875 Lease lease = leaseManager.getLease(holder); 2876 2877 if (!force && lease != null) { 2878 Lease leaseFile = leaseManager.getLeaseByPath(src); 2879 if (leaseFile != null && leaseFile.equals(lease)) { 2880 // We found the lease for this file but the original 2881 // holder is trying to obtain it again. 2882 throw new AlreadyBeingCreatedException( 2883 op.getExceptionMessage(src, holder, clientMachine, 2884 holder + " is already the current lease holder.")); 2885 } 2886 } 2887 // 2888 // Find the original holder. 2889 // 2890 FileUnderConstructionFeature uc = file.getFileUnderConstructionFeature(); 2891 String clientName = uc.getClientName(); 2892 lease = leaseManager.getLease(clientName); 2893 if (lease == null) { 2894 throw new AlreadyBeingCreatedException( 2895 op.getExceptionMessage(src, holder, clientMachine, 2896 "the file is under construction but no leases found.")); 2897 } 2898 if (force) { 2899 // close now: no need to wait for soft lease expiration and 2900 // close only the file src 2901 LOG.info("recoverLease: " + lease + ", src=" + src + 2902 " from client " + clientName); 2903 return internalReleaseLease(lease, src, iip, holder); 2904 } else { 2905 assert lease.getHolder().equals(clientName) : 2906 "Current lease holder " + lease.getHolder() + 2907 " does not match file creator " + clientName; 2908 // 2909 // If the original holder has not renewed in the last SOFTLIMIT 2910 // period, then start lease recovery. 2911 // 2912 if (lease.expiredSoftLimit()) { 2913 LOG.info("startFile: recover " + lease + ", src=" + src + " client " 2914 + clientName); 2915 if (internalReleaseLease(lease, src, iip, null)) { 2916 return true; 2917 } else { 2918 throw new RecoveryInProgressException( 2919 op.getExceptionMessage(src, holder, clientMachine, 2920 "lease recovery is in progress. Try again later.")); 2921 } 2922 } else { 2923 final BlockInfoContiguous lastBlock = file.getLastBlock(); 2924 if (lastBlock != null 2925 && lastBlock.getBlockUCState() == BlockUCState.UNDER_RECOVERY) { 2926 throw new RecoveryInProgressException( 2927 op.getExceptionMessage(src, holder, clientMachine, 2928 "another recovery is in progress by " 2929 + clientName + " on " + uc.getClientMachine())); 2930 } else { 2931 throw new AlreadyBeingCreatedException( 2932 op.getExceptionMessage(src, holder, clientMachine, 2933 "this file lease is currently owned by " 2934 + clientName + " on " + uc.getClientMachine())); 2935 } 2936 } 2937 } 2938 } else { 2939 return true; 2940 } 2941 } 2942 2943 /** 2944 * Append to an existing file in the namespace. 2945 */ 2946 LastBlockWithStatus appendFile(String src, String holder, 2947 String clientMachine, EnumSet<CreateFlag> flag, boolean logRetryCache) 2948 throws IOException { 2949 try { 2950 return appendFileInt(src, holder, clientMachine, 2951 flag.contains(CreateFlag.NEW_BLOCK), logRetryCache); 2952 } catch (AccessControlException e) { 2953 logAuditEvent(false, "append", src); 2954 throw e; 2955 } 2956 } 2957 2958 private LastBlockWithStatus appendFileInt(final String srcArg, String holder, 2959 String clientMachine, boolean newBlock, boolean logRetryCache) 2960 throws IOException { 2961 String src = srcArg; 2962 NameNode.stateChangeLog.debug( 2963 "DIR* NameSystem.appendFile: src={}, holder={}, clientMachine={}", 2964 src, holder, clientMachine); 2965 boolean skipSync = false; 2966 if (!supportAppends) { 2967 throw new UnsupportedOperationException( 2968 "Append is not enabled on this NameNode. Use the " + 2969 DFS_SUPPORT_APPEND_KEY + " configuration option to enable it."); 2970 } 2971 2972 LocatedBlock lb = null; 2973 HdfsFileStatus stat = null; 2974 FSPermissionChecker pc = getPermissionChecker(); 2975 byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); 2976 writeLock(); 2977 try { 2978 checkOperation(OperationCategory.WRITE); 2979 checkNameNodeSafeMode("Cannot append to file" + src); 2980 src = dir.resolvePath(pc, src, pathComponents); 2981 final INodesInPath iip = dir.getINodesInPath4Write(src); 2982 lb = appendFileInternal(pc, iip, holder, clientMachine, newBlock, 2983 logRetryCache); 2984 stat = FSDirStatAndListingOp.getFileInfo(dir, src, false, 2985 FSDirectory.isReservedRawName(srcArg), true); 2986 } catch (StandbyException se) { 2987 skipSync = true; 2988 throw se; 2989 } finally { 2990 writeUnlock(); 2991 // There might be transactions logged while trying to recover the lease. 2992 // They need to be sync'ed even when an exception was thrown. 2993 if (!skipSync) { 2994 getEditLog().logSync(); 2995 } 2996 } 2997 if (lb != null) { 2998 NameNode.stateChangeLog.debug( 2999 "DIR* NameSystem.appendFile: file {} for {} at {} block {} block" + 3000 " size {}", src, holder, clientMachine, lb.getBlock(), 3001 lb.getBlock().getNumBytes()); 3002 } 3003 logAuditEvent(true, "append", srcArg); 3004 return new LastBlockWithStatus(lb, stat); 3005 } 3006 3007 ExtendedBlock getExtendedBlock(Block blk) { 3008 return new ExtendedBlock(blockPoolId, blk); 3009 } 3010 3011 void setBlockPoolId(String bpid) { 3012 blockPoolId = bpid; 3013 blockManager.setBlockPoolId(blockPoolId); 3014 } 3015 3016 /** 3017 * The client would like to obtain an additional block for the indicated 3018 * filename (which is being written-to). Return an array that consists 3019 * of the block, plus a set of machines. The first on this list should 3020 * be where the client writes data. Subsequent items in the list must 3021 * be provided in the connection to the first datanode. 3022 * 3023 * Make sure the previous blocks have been reported by datanodes and 3024 * are replicated. Will return an empty 2-elt array if we want the 3025 * client to "try again later". 3026 */ 3027 LocatedBlock getAdditionalBlock(String src, long fileId, String clientName, 3028 ExtendedBlock previous, Set<Node> excludedNodes, 3029 List<String> favoredNodes) throws IOException { 3030 LocatedBlock[] onRetryBlock = new LocatedBlock[1]; 3031 DatanodeStorageInfo targets[] = getNewBlockTargets(src, fileId, 3032 clientName, previous, excludedNodes, favoredNodes, onRetryBlock); 3033 if (targets == null) { 3034 assert onRetryBlock[0] != null : "Retry block is null"; 3035 // This is a retry. Just return the last block. 3036 return onRetryBlock[0]; 3037 } 3038 LocatedBlock newBlock = storeAllocatedBlock( 3039 src, fileId, clientName, previous, targets); 3040 return newBlock; 3041 } 3042 3043 /** 3044 * Part I of getAdditionalBlock(). 3045 * Analyze the state of the file under read lock to determine if the client 3046 * can add a new block, detect potential retries, lease mismatches, 3047 * and minimal replication of the penultimate block. 3048 * 3049 * Generate target DataNode locations for the new block, 3050 * but do not create the new block yet. 3051 */ 3052 DatanodeStorageInfo[] getNewBlockTargets(String src, long fileId, 3053 String clientName, ExtendedBlock previous, Set<Node> excludedNodes, 3054 List<String> favoredNodes, LocatedBlock[] onRetryBlock) throws IOException { 3055 final long blockSize; 3056 final int replication; 3057 final byte storagePolicyID; 3058 Node clientNode = null; 3059 String clientMachine = null; 3060 3061 NameNode.stateChangeLog.debug("BLOCK* getAdditionalBlock: {} inodeId {}" + 3062 " for {}", src, fileId, clientName); 3063 3064 checkOperation(OperationCategory.READ); 3065 byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); 3066 FSPermissionChecker pc = getPermissionChecker(); 3067 readLock(); 3068 try { 3069 checkOperation(OperationCategory.READ); 3070 src = dir.resolvePath(pc, src, pathComponents); 3071 FileState fileState = analyzeFileState( 3072 src, fileId, clientName, previous, onRetryBlock); 3073 if (onRetryBlock[0] != null && onRetryBlock[0].getLocations().length > 0) { 3074 // This is a retry. No need to generate new locations. 3075 // Use the last block if it has locations. 3076 return null; 3077 } 3078 3079 final INodeFile pendingFile = fileState.inode; 3080 if (!checkFileProgress(src, pendingFile, false)) { 3081 throw new NotReplicatedYetException("Not replicated yet: " + src); 3082 } 3083 src = fileState.path; 3084 3085 if (pendingFile.getBlocks().length >= maxBlocksPerFile) { 3086 throw new IOException("File has reached the limit on maximum number of" 3087 + " blocks (" + DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY 3088 + "): " + pendingFile.getBlocks().length + " >= " 3089 + maxBlocksPerFile); 3090 } 3091 blockSize = pendingFile.getPreferredBlockSize(); 3092 clientMachine = pendingFile.getFileUnderConstructionFeature() 3093 .getClientMachine(); 3094 clientNode = blockManager.getDatanodeManager().getDatanodeByHost( 3095 clientMachine); 3096 replication = pendingFile.getFileReplication(); 3097 storagePolicyID = pendingFile.getStoragePolicyID(); 3098 } finally { 3099 readUnlock(); 3100 } 3101 3102 if (clientNode == null) { 3103 clientNode = getClientNode(clientMachine); 3104 } 3105 3106 // choose targets for the new block to be allocated. 3107 return getBlockManager().chooseTarget4NewBlock( 3108 src, replication, clientNode, excludedNodes, blockSize, favoredNodes, 3109 storagePolicyID); 3110 } 3111 3112 /** 3113 * Part II of getAdditionalBlock(). 3114 * Should repeat the same analysis of the file state as in Part 1, 3115 * but under the write lock. 3116 * If the conditions still hold, then allocate a new block with 3117 * the new targets, add it to the INode and to the BlocksMap. 3118 */ 3119 LocatedBlock storeAllocatedBlock(String src, long fileId, String clientName, 3120 ExtendedBlock previous, DatanodeStorageInfo[] targets) throws IOException { 3121 Block newBlock = null; 3122 long offset; 3123 checkOperation(OperationCategory.WRITE); 3124 waitForLoadingFSImage(); 3125 writeLock(); 3126 try { 3127 checkOperation(OperationCategory.WRITE); 3128 // Run the full analysis again, since things could have changed 3129 // while chooseTarget() was executing. 3130 LocatedBlock[] onRetryBlock = new LocatedBlock[1]; 3131 FileState fileState = 3132 analyzeFileState(src, fileId, clientName, previous, onRetryBlock); 3133 final INodeFile pendingFile = fileState.inode; 3134 src = fileState.path; 3135 3136 if (onRetryBlock[0] != null) { 3137 if (onRetryBlock[0].getLocations().length > 0) { 3138 // This is a retry. Just return the last block if having locations. 3139 return onRetryBlock[0]; 3140 } else { 3141 // add new chosen targets to already allocated block and return 3142 BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock(); 3143 ((BlockInfoContiguousUnderConstruction) lastBlockInFile) 3144 .setExpectedLocations(targets); 3145 offset = pendingFile.computeFileSize(); 3146 return makeLocatedBlock(lastBlockInFile, targets, offset); 3147 } 3148 } 3149 3150 // commit the last block and complete it if it has minimum replicas 3151 commitOrCompleteLastBlock(pendingFile, fileState.iip, 3152 ExtendedBlock.getLocalBlock(previous)); 3153 3154 // allocate new block, record block locations in INode. 3155 newBlock = createNewBlock(); 3156 INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile); 3157 saveAllocatedBlock(src, inodesInPath, newBlock, targets); 3158 3159 persistNewBlock(src, pendingFile); 3160 offset = pendingFile.computeFileSize(); 3161 } finally { 3162 writeUnlock(); 3163 } 3164 getEditLog().logSync(); 3165 3166 // Return located block 3167 return makeLocatedBlock(newBlock, targets, offset); 3168 } 3169 3170 /* 3171 * Resolve clientmachine address to get a network location path 3172 */ 3173 private Node getClientNode(String clientMachine) { 3174 List<String> hosts = new ArrayList<String>(1); 3175 hosts.add(clientMachine); 3176 List<String> rName = getBlockManager().getDatanodeManager() 3177 .resolveNetworkLocation(hosts); 3178 Node clientNode = null; 3179 if (rName != null) { 3180 // Able to resolve clientMachine mapping. 3181 // Create a temp node to findout the rack local nodes 3182 clientNode = new NodeBase(rName.get(0) + NodeBase.PATH_SEPARATOR_STR 3183 + clientMachine); 3184 } 3185 return clientNode; 3186 } 3187 3188 static class FileState { 3189 public final INodeFile inode; 3190 public final String path; 3191 public final INodesInPath iip; 3192 3193 public FileState(INodeFile inode, String fullPath, INodesInPath iip) { 3194 this.inode = inode; 3195 this.path = fullPath; 3196 this.iip = iip; 3197 } 3198 } 3199 3200 FileState analyzeFileState(String src, 3201 long fileId, 3202 String clientName, 3203 ExtendedBlock previous, 3204 LocatedBlock[] onRetryBlock) 3205 throws IOException { 3206 assert hasReadLock(); 3207 3208 checkBlock(previous); 3209 onRetryBlock[0] = null; 3210 checkNameNodeSafeMode("Cannot add block to " + src); 3211 3212 // have we exceeded the configured limit of fs objects. 3213 checkFsObjectLimit(); 3214 3215 Block previousBlock = ExtendedBlock.getLocalBlock(previous); 3216 final INode inode; 3217 final INodesInPath iip; 3218 if (fileId == INodeId.GRANDFATHER_INODE_ID) { 3219 // Older clients may not have given us an inode ID to work with. 3220 // In this case, we have to try to resolve the path and hope it 3221 // hasn't changed or been deleted since the file was opened for write. 3222 iip = dir.getINodesInPath4Write(src); 3223 inode = iip.getLastINode(); 3224 } else { 3225 // Newer clients pass the inode ID, so we can just get the inode 3226 // directly. 3227 inode = dir.getInode(fileId); 3228 iip = INodesInPath.fromINode(inode); 3229 if (inode != null) { 3230 src = iip.getPath(); 3231 } 3232 } 3233 final INodeFile pendingFile = checkLease(src, clientName, inode, fileId); 3234 BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock(); 3235 if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) { 3236 // The block that the client claims is the current last block 3237 // doesn't match up with what we think is the last block. There are 3238 // four possibilities: 3239 // 1) This is the first block allocation of an append() pipeline 3240 // which started appending exactly at or exceeding the block boundary. 3241 // In this case, the client isn't passed the previous block, 3242 // so it makes the allocateBlock() call with previous=null. 3243 // We can distinguish this since the last block of the file 3244 // will be exactly a full block. 3245 // 2) This is a retry from a client that missed the response of a 3246 // prior getAdditionalBlock() call, perhaps because of a network 3247 // timeout, or because of an HA failover. In that case, we know 3248 // by the fact that the client is re-issuing the RPC that it 3249 // never began to write to the old block. Hence it is safe to 3250 // to return the existing block. 3251 // 3) This is an entirely bogus request/bug -- we should error out 3252 // rather than potentially appending a new block with an empty 3253 // one in the middle, etc 3254 // 4) This is a retry from a client that timed out while 3255 // the prior getAdditionalBlock() is still being processed, 3256 // currently working on chooseTarget(). 3257 // There are no means to distinguish between the first and 3258 // the second attempts in Part I, because the first one hasn't 3259 // changed the namesystem state yet. 3260 // We run this analysis again in Part II where case 4 is impossible. 3261 3262 BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock(); 3263 if (previous == null && 3264 lastBlockInFile != null && 3265 lastBlockInFile.getNumBytes() >= pendingFile.getPreferredBlockSize() && 3266 lastBlockInFile.isComplete()) { 3267 // Case 1 3268 NameNode.stateChangeLog.debug( 3269 "BLOCK* NameSystem.allocateBlock: handling block allocation" + 3270 " writing to a file with a complete previous block: src={}" + 3271 " lastBlock={}", src, lastBlockInFile); 3272 } else if (Block.matchingIdAndGenStamp(penultimateBlock, previousBlock)) { 3273 if (lastBlockInFile.getNumBytes() != 0) { 3274 throw new IOException( 3275 "Request looked like a retry to allocate block " + 3276 lastBlockInFile + " but it already contains " + 3277 lastBlockInFile.getNumBytes() + " bytes"); 3278 } 3279 3280 // Case 2 3281 // Return the last block. 3282 NameNode.stateChangeLog.info("BLOCK* allocateBlock: " + 3283 "caught retry for allocation of a new block in " + 3284 src + ". Returning previously allocated block " + lastBlockInFile); 3285 long offset = pendingFile.computeFileSize(); 3286 onRetryBlock[0] = makeLocatedBlock(lastBlockInFile, 3287 ((BlockInfoContiguousUnderConstruction)lastBlockInFile).getExpectedStorageLocations(), 3288 offset); 3289 return new FileState(pendingFile, src, iip); 3290 } else { 3291 // Case 3 3292 throw new IOException("Cannot allocate block in " + src + ": " + 3293 "passed 'previous' block " + previous + " does not match actual " + 3294 "last block in file " + lastBlockInFile); 3295 } 3296 } 3297 return new FileState(pendingFile, src, iip); 3298 } 3299 3300 LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs, 3301 long offset) throws IOException { 3302 LocatedBlock lBlk = new LocatedBlock( 3303 getExtendedBlock(blk), locs, offset, false); 3304 getBlockManager().setBlockToken( 3305 lBlk, BlockTokenSecretManager.AccessMode.WRITE); 3306 return lBlk; 3307 } 3308 3309 /** @see ClientProtocol#getAdditionalDatanode */ 3310 LocatedBlock getAdditionalDatanode(String src, long fileId, 3311 final ExtendedBlock blk, final DatanodeInfo[] existings, 3312 final String[] storageIDs, 3313 final Set<Node> excludes, 3314 final int numAdditionalNodes, final String clientName 3315 ) throws IOException { 3316 //check if the feature is enabled 3317 dtpReplaceDatanodeOnFailure.checkEnabled(); 3318 3319 Node clientnode = null; 3320 String clientMachine; 3321 final long preferredblocksize; 3322 final byte storagePolicyID; 3323 final List<DatanodeStorageInfo> chosen; 3324 checkOperation(OperationCategory.READ); 3325 byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); 3326 FSPermissionChecker pc = getPermissionChecker(); 3327 readLock(); 3328 try { 3329 checkOperation(OperationCategory.READ); 3330 //check safe mode 3331 checkNameNodeSafeMode("Cannot add datanode; src=" + src + ", blk=" + blk); 3332 src = dir.resolvePath(pc, src, pathComponents); 3333 3334 //check lease 3335 final INode inode; 3336 if (fileId == INodeId.GRANDFATHER_INODE_ID) { 3337 // Older clients may not have given us an inode ID to work with. 3338 // In this case, we have to try to resolve the path and hope it 3339 // hasn't changed or been deleted since the file was opened for write. 3340 inode = dir.getINode(src); 3341 } else { 3342 inode = dir.getInode(fileId); 3343 if (inode != null) src = inode.getFullPathName(); 3344 } 3345 final INodeFile file = checkLease(src, clientName, inode, fileId); 3346 clientMachine = file.getFileUnderConstructionFeature().getClientMachine(); 3347 clientnode = blockManager.getDatanodeManager().getDatanodeByHost(clientMachine); 3348 preferredblocksize = file.getPreferredBlockSize(); 3349 storagePolicyID = file.getStoragePolicyID(); 3350 3351 //find datanode storages 3352 final DatanodeManager dm = blockManager.getDatanodeManager(); 3353 chosen = Arrays.asList(dm.getDatanodeStorageInfos(existings, storageIDs, 3354 "src=%s, fileId=%d, blk=%s, clientName=%s, clientMachine=%s", 3355 src, fileId, blk, clientName, clientMachine)); 3356 } finally { 3357 readUnlock(); 3358 } 3359 3360 if (clientnode == null) { 3361 clientnode = getClientNode(clientMachine); 3362 } 3363 3364 // choose new datanodes. 3365 final DatanodeStorageInfo[] targets = blockManager.chooseTarget4AdditionalDatanode( 3366 src, numAdditionalNodes, clientnode, chosen, 3367 excludes, preferredblocksize, storagePolicyID); 3368 final LocatedBlock lb = new LocatedBlock(blk, targets); 3369 blockManager.setBlockToken(lb, AccessMode.COPY); 3370 return lb; 3371 } 3372 3373 /** 3374 * The client would like to let go of the given block 3375 */ 3376 boolean abandonBlock(ExtendedBlock b, long fileId, String src, String holder) 3377 throws IOException { 3378 NameNode.stateChangeLog.debug( 3379 "BLOCK* NameSystem.abandonBlock: {} of file {}", b, src); 3380 checkOperation(OperationCategory.WRITE); 3381 byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); 3382 FSPermissionChecker pc = getPermissionChecker(); 3383 waitForLoadingFSImage(); 3384 writeLock(); 3385 try { 3386 checkOperation(OperationCategory.WRITE); 3387 checkNameNodeSafeMode("Cannot abandon block " + b + " for file" + src); 3388 src = dir.resolvePath(pc, src, pathComponents); 3389 3390 final INode inode; 3391 final INodesInPath iip; 3392 if (fileId == INodeId.GRANDFATHER_INODE_ID) { 3393 // Older clients may not have given us an inode ID to work with. 3394 // In this case, we have to try to resolve the path and hope it 3395 // hasn't changed or been deleted since the file was opened for write. 3396 iip = dir.getINodesInPath(src, true); 3397 inode = iip.getLastINode(); 3398 } else { 3399 inode = dir.getInode(fileId); 3400 iip = INodesInPath.fromINode(inode); 3401 if (inode != null) { 3402 src = iip.getPath(); 3403 } 3404 } 3405 final INodeFile file = checkLease(src, holder, inode, fileId); 3406 3407 // Remove the block from the pending creates list 3408 boolean removed = dir.removeBlock(src, iip, file, 3409 ExtendedBlock.getLocalBlock(b)); 3410 if (!removed) { 3411 return true; 3412 } 3413 NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: {} is " + 3414 "removed from pendingCreates", b); 3415 persistBlocks(src, file, false); 3416 } finally { 3417 writeUnlock(); 3418 } 3419 getEditLog().logSync(); 3420 3421 return true; 3422 } 3423 3424 private INodeFile checkLease(String src, String holder, INode inode, 3425 long fileId) throws LeaseExpiredException, FileNotFoundException { 3426 assert hasReadLock(); 3427 final String ident = src + " (inode " + fileId + ")"; 3428 if (inode == null) { 3429 Lease lease = leaseManager.getLease(holder); 3430 throw new LeaseExpiredException( 3431 "No lease on " + ident + ": File does not exist. " 3432 + (lease != null ? lease.toString() 3433 : "Holder " + holder + " does not have any open files.")); 3434 } 3435 if (!inode.isFile()) { 3436 Lease lease = leaseManager.getLease(holder); 3437 throw new LeaseExpiredException( 3438 "No lease on " + ident + ": INode is not a regular file. " 3439 + (lease != null ? lease.toString() 3440 : "Holder " + holder + " does not have any open files.")); 3441 } 3442 final INodeFile file = inode.asFile(); 3443 if (!file.isUnderConstruction()) { 3444 Lease lease = leaseManager.getLease(holder); 3445 throw new LeaseExpiredException( 3446 "No lease on " + ident + ": File is not open for writing. " 3447 + (lease != null ? lease.toString() 3448 : "Holder " + holder + " does not have any open files.")); 3449 } 3450 // No further modification is allowed on a deleted file. 3451 // A file is considered deleted, if it is not in the inodeMap or is marked 3452 // as deleted in the snapshot feature. 3453 if (isFileDeleted(file)) { 3454 throw new FileNotFoundException(src); 3455 } 3456 String clientName = file.getFileUnderConstructionFeature().getClientName(); 3457 if (holder != null && !clientName.equals(holder)) { 3458 throw new LeaseExpiredException("Lease mismatch on " + ident + 3459 " owned by " + clientName + " but is accessed by " + holder); 3460 } 3461 return file; 3462 } 3463 3464 /** 3465 * Complete in-progress write to the given file. 3466 * @return true if successful, false if the client should continue to retry 3467 * (e.g if not all blocks have reached minimum replication yet) 3468 * @throws IOException on error (eg lease mismatch, file not open, file deleted) 3469 */ 3470 boolean completeFile(final String srcArg, String holder, 3471 ExtendedBlock last, long fileId) 3472 throws SafeModeException, UnresolvedLinkException, IOException { 3473 String src = srcArg; 3474 NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: {} for {}", 3475 src, holder); 3476 checkBlock(last); 3477 boolean success = false; 3478 checkOperation(OperationCategory.WRITE); 3479 byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); 3480 FSPermissionChecker pc = getPermissionChecker(); 3481 waitForLoadingFSImage(); 3482 writeLock(); 3483 try { 3484 checkOperation(OperationCategory.WRITE); 3485 checkNameNodeSafeMode("Cannot complete file " + src); 3486 src = dir.resolvePath(pc, src, pathComponents); 3487 success = completeFileInternal(src, holder, 3488 ExtendedBlock.getLocalBlock(last), fileId); 3489 } finally { 3490 writeUnlock(); 3491 } 3492 getEditLog().logSync(); 3493 if (success) { 3494 NameNode.stateChangeLog.info("DIR* completeFile: " + srcArg 3495 + " is closed by " + holder); 3496 } 3497 return success; 3498 } 3499 3500 private boolean completeFileInternal(String src, String holder, Block last, 3501 long fileId) throws IOException { 3502 assert hasWriteLock(); 3503 final INodeFile pendingFile; 3504 final INodesInPath iip; 3505 INode inode = null; 3506 try { 3507 if (fileId == INodeId.GRANDFATHER_INODE_ID) { 3508 // Older clients may not have given us an inode ID to work with. 3509 // In this case, we have to try to resolve the path and hope it 3510 // hasn't changed or been deleted since the file was opened for write. 3511 iip = dir.getINodesInPath(src, true); 3512 inode = iip.getLastINode(); 3513 } else { 3514 inode = dir.getInode(fileId); 3515 iip = INodesInPath.fromINode(inode); 3516 if (inode != null) { 3517 src = iip.getPath(); 3518 } 3519 } 3520 pendingFile = checkLease(src, holder, inode, fileId); 3521 } catch (LeaseExpiredException lee) { 3522 if (inode != null && inode.isFile() && 3523 !inode.asFile().isUnderConstruction()) { 3524 // This could be a retry RPC - i.e the client tried to close 3525 // the file, but missed the RPC response. Thus, it is trying 3526 // again to close the file. If the file still exists and 3527 // the client's view of the last block matches the actual 3528 // last block, then we'll treat it as a successful close. 3529 // See HDFS-3031. 3530 final Block realLastBlock = inode.asFile().getLastBlock(); 3531 if (Block.matchingIdAndGenStamp(last, realLastBlock)) { 3532 NameNode.stateChangeLog.info("DIR* completeFile: " + 3533 "request from " + holder + " to complete inode " + fileId + 3534 "(" + src + ") which is already closed. But, it appears to be " + 3535 "an RPC retry. Returning success"); 3536 return true; 3537 } 3538 } 3539 throw lee; 3540 } 3541 // Check the state of the penultimate block. It should be completed 3542 // before attempting to complete the last one. 3543 if (!checkFileProgress(src, pendingFile, false)) { 3544 return false; 3545 } 3546 3547 // commit the last block and complete it if it has minimum replicas 3548 commitOrCompleteLastBlock(pendingFile, iip, last); 3549 3550 if (!checkFileProgress(src, pendingFile, true)) { 3551 return false; 3552 } 3553 3554 finalizeINodeFileUnderConstruction(src, pendingFile, 3555 Snapshot.CURRENT_STATE_ID); 3556 return true; 3557 } 3558 3559 /** 3560 * Save allocated block at the given pending filename 3561 * 3562 * @param src path to the file 3563 * @param inodesInPath representing each of the components of src. 3564 * The last INode is the INode for {@code src} file. 3565 * @param newBlock newly allocated block to be save 3566 * @param targets target datanodes where replicas of the new block is placed 3567 * @throws QuotaExceededException If addition of block exceeds space quota 3568 */ 3569 BlockInfoContiguous saveAllocatedBlock(String src, INodesInPath inodesInPath, 3570 Block newBlock, DatanodeStorageInfo[] targets) 3571 throws IOException { 3572 assert hasWriteLock(); 3573 BlockInfoContiguous b = dir.addBlock(src, inodesInPath, newBlock, targets); 3574 NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src); 3575 DatanodeStorageInfo.incrementBlocksScheduled(targets); 3576 return b; 3577 } 3578 3579 /** 3580 * Create new block with a unique block id and a new generation stamp. 3581 */ 3582 Block createNewBlock() throws IOException { 3583 assert hasWriteLock(); 3584 Block b = new Block(nextBlockId(), 0, 0); 3585 // Increment the generation stamp for every new block. 3586 b.setGenerationStamp(nextGenerationStamp(false)); 3587 return b; 3588 } 3589 3590 /** 3591 * Check that the indicated file's blocks are present and 3592 * replicated. If not, return false. If checkall is true, then check 3593 * all blocks, otherwise check only penultimate block. 3594 */ 3595 boolean checkFileProgress(String src, INodeFile v, boolean checkall) { 3596 if (checkall) { 3597 // check all blocks of the file. 3598 for (BlockInfoContiguous block: v.getBlocks()) { 3599 if (!isCompleteBlock(src, block, blockManager.minReplication)) { 3600 return false; 3601 } 3602 } 3603 } else { 3604 // check the penultimate block of this file 3605 BlockInfoContiguous b = v.getPenultimateBlock(); 3606 if (b != null 3607 && !isCompleteBlock(src, b, blockManager.minReplication)) { 3608 return false; 3609 } 3610 } 3611 return true; 3612 } 3613 3614 private static boolean isCompleteBlock(String src, BlockInfoContiguous b, int minRepl) { 3615 if (!b.isComplete()) { 3616 final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)b; 3617 final int numNodes = b.numNodes(); 3618 LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = " 3619 + uc.getBlockUCState() + ", replication# = " + numNodes 3620 + (numNodes < minRepl? " < ": " >= ") 3621 + " minimum = " + minRepl + ") in file " + src); 3622 return false; 3623 } 3624 return true; 3625 } 3626 3627 //////////////////////////////////////////////////////////////// 3628 // Here's how to handle block-copy failure during client write: 3629 // -- As usual, the client's write should result in a streaming 3630 // backup write to a k-machine sequence. 3631 // -- If one of the backup machines fails, no worries. Fail silently. 3632 // -- Before client is allowed to close and finalize file, make sure 3633 // that the blocks are backed up. Namenode may have to issue specific backup 3634 // commands to make up for earlier datanode failures. Once all copies 3635 // are made, edit namespace and return to client. 3636 //////////////////////////////////////////////////////////////// 3637 3638 /** 3639 * Change the indicated filename. 3640 * @deprecated Use {@link #renameTo(String, String, boolean, 3641 * Options.Rename...)} instead. 3642 */ 3643 @Deprecated 3644 boolean renameTo(String src, String dst, boolean logRetryCache) 3645 throws IOException { 3646 waitForLoadingFSImage(); 3647 FSDirRenameOp.RenameOldResult ret = null; 3648 writeLock(); 3649 try { 3650 checkOperation(OperationCategory.WRITE); 3651 checkNameNodeSafeMode("Cannot rename " + src); 3652 ret = FSDirRenameOp.renameToInt(dir, src, dst, logRetryCache); 3653 } catch (AccessControlException e) { 3654 logAuditEvent(false, "rename", src, dst, null); 3655 throw e; 3656 } finally { 3657 writeUnlock(); 3658 } 3659 boolean success = ret != null && ret.success; 3660 if (success) { 3661 getEditLog().logSync(); 3662 } 3663 logAuditEvent(success, "rename", src, dst, 3664 ret == null ? null : ret.auditStat); 3665 return success; 3666 } 3667 3668 void renameTo(final String src, final String dst, 3669 boolean logRetryCache, Options.Rename... options) 3670 throws IOException { 3671 waitForLoadingFSImage(); 3672 Map.Entry<BlocksMapUpdateInfo, HdfsFileStatus> res = null; 3673 writeLock(); 3674 try { 3675 checkOperation(OperationCategory.WRITE); 3676 checkNameNodeSafeMode("Cannot rename " + src); 3677 res = FSDirRenameOp.renameToInt(dir, src, dst, logRetryCache, options); 3678 } catch (AccessControlException e) { 3679 logAuditEvent(false, "rename (options=" + Arrays.toString(options) + 3680 ")", src, dst, null); 3681 throw e; 3682 } finally { 3683 writeUnlock(); 3684 } 3685 3686 getEditLog().logSync(); 3687 3688 BlocksMapUpdateInfo collectedBlocks = res.getKey(); 3689 HdfsFileStatus auditStat = res.getValue(); 3690 if (!collectedBlocks.getToDeleteList().isEmpty()) { 3691 removeBlocks(collectedBlocks); 3692 collectedBlocks.clear(); 3693 } 3694 3695 logAuditEvent(true, "rename (options=" + Arrays.toString(options) + 3696 ")", src, dst, auditStat); 3697 } 3698 3699 /** 3700 * Remove the indicated file from namespace. 3701 * 3702 * @see ClientProtocol#delete(String, boolean) for detailed description and 3703 * description of exceptions 3704 */ 3705 boolean delete(String src, boolean recursive, boolean logRetryCache) 3706 throws IOException { 3707 waitForLoadingFSImage(); 3708 BlocksMapUpdateInfo toRemovedBlocks = null; 3709 writeLock(); 3710 boolean ret = false; 3711 try { 3712 checkOperation(OperationCategory.WRITE); 3713 checkNameNodeSafeMode("Cannot delete " + src); 3714 toRemovedBlocks = FSDirDeleteOp.delete( 3715 this, src, recursive, logRetryCache); 3716 ret = toRemovedBlocks != null; 3717 } catch (AccessControlException e) { 3718 logAuditEvent(false, "delete", src); 3719 throw e; 3720 } finally { 3721 writeUnlock(); 3722 } 3723 getEditLog().logSync(); 3724 if (toRemovedBlocks != null) { 3725 removeBlocks(toRemovedBlocks); // Incremental deletion of blocks 3726 } 3727 logAuditEvent(true, "delete", src); 3728 return ret; 3729 } 3730 3731 FSPermissionChecker getPermissionChecker() 3732 throws AccessControlException { 3733 return dir.getPermissionChecker(); 3734 } 3735 3736 /** 3737 * From the given list, incrementally remove the blocks from blockManager 3738 * Writelock is dropped and reacquired every BLOCK_DELETION_INCREMENT to 3739 * ensure that other waiters on the lock can get in. See HDFS-2938 3740 * 3741 * @param blocks 3742 * An instance of {@link BlocksMapUpdateInfo} which contains a list 3743 * of blocks that need to be removed from blocksMap 3744 */ 3745 void removeBlocks(BlocksMapUpdateInfo blocks) { 3746 List<Block> toDeleteList = blocks.getToDeleteList(); 3747 Iterator<Block> iter = toDeleteList.iterator(); 3748 while (iter.hasNext()) { 3749 writeLock(); 3750 try { 3751 for (int i = 0; i < BLOCK_DELETION_INCREMENT && iter.hasNext(); i++) { 3752 blockManager.removeBlock(iter.next()); 3753 } 3754 } finally { 3755 writeUnlock(); 3756 } 3757 } 3758 } 3759 3760 /** 3761 * Remove leases and inodes related to a given path 3762 * @param src The given path 3763 * @param removedINodes Containing the list of inodes to be removed from 3764 * inodesMap 3765 * @param acquireINodeMapLock Whether to acquire the lock for inode removal 3766 */ 3767 void removeLeasesAndINodes(String src, List<INode> removedINodes, 3768 final boolean acquireINodeMapLock) { 3769 assert hasWriteLock(); 3770 leaseManager.removeLeaseWithPrefixPath(src); 3771 // remove inodes from inodesMap 3772 if (removedINodes != null) { 3773 if (acquireINodeMapLock) { 3774 dir.writeLock(); 3775 } 3776 try { 3777 dir.removeFromInodeMap(removedINodes); 3778 } finally { 3779 if (acquireINodeMapLock) { 3780 dir.writeUnlock(); 3781 } 3782 } 3783 removedINodes.clear(); 3784 } 3785 } 3786 3787 /** 3788 * Removes the blocks from blocksmap and updates the safemode blocks total 3789 * 3790 * @param blocks 3791 * An instance of {@link BlocksMapUpdateInfo} which contains a list 3792 * of blocks that need to be removed from blocksMap 3793 */ 3794 void removeBlocksAndUpdateSafemodeTotal(BlocksMapUpdateInfo blocks) { 3795 assert hasWriteLock(); 3796 // In the case that we are a Standby tailing edits from the 3797 // active while in safe-mode, we need to track the total number 3798 // of blocks and safe blocks in the system. 3799 boolean trackBlockCounts = isSafeModeTrackingBlocks(); 3800 int numRemovedComplete = 0, numRemovedSafe = 0; 3801 3802 for (Block b : blocks.getToDeleteList()) { 3803 if (trackBlockCounts) { 3804 BlockInfoContiguous bi = getStoredBlock(b); 3805 if (bi.isComplete()) { 3806 numRemovedComplete++; 3807 if (bi.numNodes() >= blockManager.minReplication) { 3808 numRemovedSafe++; 3809 } 3810 } 3811 } 3812 blockManager.removeBlock(b); 3813 } 3814 if (trackBlockCounts) { 3815 if (LOG.isDebugEnabled()) { 3816 LOG.debug("Adjusting safe-mode totals for deletion." 3817 + "decreasing safeBlocks by " + numRemovedSafe 3818 + ", totalBlocks by " + numRemovedComplete); 3819 } 3820 adjustSafeModeBlockTotals(-numRemovedSafe, -numRemovedComplete); 3821 } 3822 } 3823 3824 /** 3825 * @see SafeModeInfo#shouldIncrementallyTrackBlocks 3826 */ 3827 private boolean isSafeModeTrackingBlocks() { 3828 if (!haEnabled) { 3829 // Never track blocks incrementally in non-HA code. 3830 return false; 3831 } 3832 SafeModeInfo sm = this.safeMode; 3833 return sm != null && sm.shouldIncrementallyTrackBlocks(); 3834 } 3835 3836 /** 3837 * Get the file info for a specific file. 3838 * 3839 * @param src The string representation of the path to the file 3840 * @param resolveLink whether to throw UnresolvedLinkException 3841 * if src refers to a symlink 3842 * 3843 * @throws AccessControlException if access is denied 3844 * @throws UnresolvedLinkException if a symlink is encountered. 3845 * 3846 * @return object containing information regarding the file 3847 * or null if file not found 3848 * @throws StandbyException 3849 */ 3850 HdfsFileStatus getFileInfo(final String src, boolean resolveLink) 3851 throws IOException { 3852 checkOperation(OperationCategory.READ); 3853 HdfsFileStatus stat = null; 3854 readLock(); 3855 try { 3856 checkOperation(OperationCategory.READ); 3857 stat = FSDirStatAndListingOp.getFileInfo(dir, src, resolveLink); 3858 } catch (AccessControlException e) { 3859 logAuditEvent(false, "getfileinfo", src); 3860 throw e; 3861 } finally { 3862 readUnlock(); 3863 } 3864 logAuditEvent(true, "getfileinfo", src); 3865 return stat; 3866 } 3867 3868 /** 3869 * Returns true if the file is closed 3870 */ 3871 boolean isFileClosed(final String src) throws IOException { 3872 checkOperation(OperationCategory.READ); 3873 readLock(); 3874 try { 3875 checkOperation(OperationCategory.READ); 3876 return FSDirStatAndListingOp.isFileClosed(dir, src); 3877 } catch (AccessControlException e) { 3878 logAuditEvent(false, "isFileClosed", src); 3879 throw e; 3880 } finally { 3881 readUnlock(); 3882 } 3883 } 3884 3885 /** 3886 * Create all the necessary directories 3887 */ 3888 boolean mkdirs(String src, PermissionStatus permissions, 3889 boolean createParent) throws IOException { 3890 HdfsFileStatus auditStat = null; 3891 checkOperation(OperationCategory.WRITE); 3892 writeLock(); 3893 try { 3894 checkOperation(OperationCategory.WRITE); 3895 checkNameNodeSafeMode("Cannot create directory " + src); 3896 auditStat = FSDirMkdirOp.mkdirs(this, src, permissions, createParent); 3897 } catch (AccessControlException e) { 3898 logAuditEvent(false, "mkdirs", src); 3899 throw e; 3900 } finally { 3901 writeUnlock(); 3902 } 3903 getEditLog().logSync(); 3904 logAuditEvent(true, "mkdirs", src, null, auditStat); 3905 return true; 3906 } 3907 3908 /** 3909 * Get the content summary for a specific file/dir. 3910 * 3911 * @param src The string representation of the path to the file 3912 * 3913 * @throws AccessControlException if access is denied 3914 * @throws UnresolvedLinkException if a symlink is encountered. 3915 * @throws FileNotFoundException if no file exists 3916 * @throws StandbyException 3917 * @throws IOException for issues with writing to the audit log 3918 * 3919 * @return object containing information regarding the file 3920 * or null if file not found 3921 */ 3922 ContentSummary getContentSummary(final String src) throws IOException { 3923 checkOperation(OperationCategory.READ); 3924 readLock(); 3925 boolean success = true; 3926 try { 3927 checkOperation(OperationCategory.READ); 3928 return FSDirStatAndListingOp.getContentSummary(dir, src); 3929 } catch (AccessControlException ace) { 3930 success = false; 3931 throw ace; 3932 } finally { 3933 readUnlock(); 3934 logAuditEvent(success, "contentSummary", src); 3935 } 3936 } 3937 3938 /** 3939 * Set the namespace quota and storage space quota for a directory. 3940 * See {@link ClientProtocol#setQuota(String, long, long, StorageType)} for the 3941 * contract. 3942 * 3943 * Note: This does not support ".inodes" relative path. 3944 */ 3945 void setQuota(String src, long nsQuota, long ssQuota, StorageType type) 3946 throws IOException { 3947 checkOperation(OperationCategory.WRITE); 3948 writeLock(); 3949 boolean success = false; 3950 try { 3951 checkOperation(OperationCategory.WRITE); 3952 checkNameNodeSafeMode("Cannot set quota on " + src); 3953 FSDirAttrOp.setQuota(dir, src, nsQuota, ssQuota, type); 3954 success = true; 3955 } finally { 3956 writeUnlock(); 3957 if (success) { 3958 getEditLog().logSync(); 3959 } 3960 logAuditEvent(success, "setQuota", src); 3961 } 3962 } 3963 3964 /** Persist all metadata about this file. 3965 * @param src The string representation of the path 3966 * @param fileId The inode ID that we're fsyncing. Older clients will pass 3967 * INodeId.GRANDFATHER_INODE_ID here. 3968 * @param clientName The string representation of the client 3969 * @param lastBlockLength The length of the last block 3970 * under construction reported from client. 3971 * @throws IOException if path does not exist 3972 */ 3973 void fsync(String src, long fileId, String clientName, long lastBlockLength) 3974 throws IOException { 3975 NameNode.stateChangeLog.info("BLOCK* fsync: " + src + " for " + clientName); 3976 checkOperation(OperationCategory.WRITE); 3977 byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); 3978 3979 FSPermissionChecker pc = getPermissionChecker(); 3980 waitForLoadingFSImage(); 3981 writeLock(); 3982 try { 3983 checkOperation(OperationCategory.WRITE); 3984 checkNameNodeSafeMode("Cannot fsync file " + src); 3985 src = dir.resolvePath(pc, src, pathComponents); 3986 final INode inode; 3987 if (fileId == INodeId.GRANDFATHER_INODE_ID) { 3988 // Older clients may not have given us an inode ID to work with. 3989 // In this case, we have to try to resolve the path and hope it 3990 // hasn't changed or been deleted since the file was opened for write. 3991 inode = dir.getINode(src); 3992 } else { 3993 inode = dir.getInode(fileId); 3994 if (inode != null) src = inode.getFullPathName(); 3995 } 3996 final INodeFile pendingFile = checkLease(src, clientName, inode, fileId); 3997 if (lastBlockLength > 0) { 3998 pendingFile.getFileUnderConstructionFeature().updateLengthOfLastBlock( 3999 pendingFile, lastBlockLength); 4000 } 4001 persistBlocks(src, pendingFile, false); 4002 } finally { 4003 writeUnlock(); 4004 } 4005 getEditLog().logSync(); 4006 } 4007 4008 /** 4009 * Move a file that is being written to be immutable. 4010 * @param src The filename 4011 * @param lease The lease for the client creating the file 4012 * @param recoveryLeaseHolder reassign lease to this holder if the last block 4013 * needs recovery; keep current holder if null. 4014 * @throws AlreadyBeingCreatedException if file is waiting to achieve minimal 4015 * replication;<br> 4016 * RecoveryInProgressException if lease recovery is in progress.<br> 4017 * IOException in case of an error. 4018 * @return true if file has been successfully finalized and closed or 4019 * false if block recovery has been initiated. Since the lease owner 4020 * has been changed and logged, caller should call logSync(). 4021 */ 4022 boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, 4023 String recoveryLeaseHolder) throws IOException { 4024 LOG.info("Recovering " + lease + ", src=" + src); 4025 assert !isInSafeMode(); 4026 assert hasWriteLock(); 4027 4028 final INodeFile pendingFile = iip.getLastINode().asFile(); 4029 int nrBlocks = pendingFile.numBlocks(); 4030 BlockInfoContiguous[] blocks = pendingFile.getBlocks(); 4031 4032 int nrCompleteBlocks; 4033 BlockInfoContiguous curBlock = null; 4034 for(nrCompleteBlocks = 0; nrCompleteBlocks < nrBlocks; nrCompleteBlocks++) { 4035 curBlock = blocks[nrCompleteBlocks]; 4036 if(!curBlock.isComplete()) 4037 break; 4038 assert blockManager.checkMinReplication(curBlock) : 4039 "A COMPLETE block is not minimally replicated in " + src; 4040 } 4041 4042 // If there are no incomplete blocks associated with this file, 4043 // then reap lease immediately and close the file. 4044 if(nrCompleteBlocks == nrBlocks) { 4045 finalizeINodeFileUnderConstruction(src, pendingFile, 4046 iip.getLatestSnapshotId()); 4047 NameNode.stateChangeLog.warn("BLOCK*" 4048 + " internalReleaseLease: All existing blocks are COMPLETE," 4049 + " lease removed, file closed."); 4050 return true; // closed! 4051 } 4052 4053 // Only the last and the penultimate blocks may be in non COMPLETE state. 4054 // If the penultimate block is not COMPLETE, then it must be COMMITTED. 4055 if(nrCompleteBlocks < nrBlocks - 2 || 4056 nrCompleteBlocks == nrBlocks - 2 && 4057 curBlock != null && 4058 curBlock.getBlockUCState() != BlockUCState.COMMITTED) { 4059 final String message = "DIR* NameSystem.internalReleaseLease: " 4060 + "attempt to release a create lock on " 4061 + src + " but file is already closed."; 4062 NameNode.stateChangeLog.warn(message); 4063 throw new IOException(message); 4064 } 4065 4066 // The last block is not COMPLETE, and 4067 // that the penultimate block if exists is either COMPLETE or COMMITTED 4068 final BlockInfoContiguous lastBlock = pendingFile.getLastBlock(); 4069 BlockUCState lastBlockState = lastBlock.getBlockUCState(); 4070 BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock(); 4071 4072 // If penultimate block doesn't exist then its minReplication is met 4073 boolean penultimateBlockMinReplication = penultimateBlock == null ? true : 4074 blockManager.checkMinReplication(penultimateBlock); 4075 4076 switch(lastBlockState) { 4077 case COMPLETE: 4078 assert false : "Already checked that the last block is incomplete"; 4079 break; 4080 case COMMITTED: 4081 // Close file if committed blocks are minimally replicated 4082 if(penultimateBlockMinReplication && 4083 blockManager.checkMinReplication(lastBlock)) { 4084 finalizeINodeFileUnderConstruction(src, pendingFile, 4085 iip.getLatestSnapshotId()); 4086 NameNode.stateChangeLog.warn("BLOCK*" 4087 + " internalReleaseLease: Committed blocks are minimally replicated," 4088 + " lease removed, file closed."); 4089 return true; // closed! 4090 } 4091 // Cannot close file right now, since some blocks 4092 // are not yet minimally replicated. 4093 // This may potentially cause infinite loop in lease recovery 4094 // if there are no valid replicas on data-nodes. 4095 String message = "DIR* NameSystem.internalReleaseLease: " + 4096 "Failed to release lease for file " + src + 4097 ". Committed blocks are waiting to be minimally replicated." + 4098 " Try again later."; 4099 NameNode.stateChangeLog.warn(message); 4100 throw new AlreadyBeingCreatedException(message); 4101 case UNDER_CONSTRUCTION: 4102 case UNDER_RECOVERY: 4103 final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)lastBlock; 4104 // determine if last block was intended to be truncated 4105 Block recoveryBlock = uc.getTruncateBlock(); 4106 boolean truncateRecovery = recoveryBlock != null; 4107 boolean copyOnTruncate = truncateRecovery && 4108 recoveryBlock.getBlockId() != uc.getBlockId(); 4109 assert !copyOnTruncate || 4110 recoveryBlock.getBlockId() < uc.getBlockId() && 4111 recoveryBlock.getGenerationStamp() < uc.getGenerationStamp() && 4112 recoveryBlock.getNumBytes() > uc.getNumBytes() : 4113 "wrong recoveryBlock"; 4114 4115 // setup the last block locations from the blockManager if not known 4116 if (uc.getNumExpectedLocations() == 0) { 4117 uc.setExpectedLocations(blockManager.getStorages(lastBlock)); 4118 } 4119 4120 if (uc.getNumExpectedLocations() == 0 && uc.getNumBytes() == 0) { 4121 // There is no datanode reported to this block. 4122 // may be client have crashed before writing data to pipeline. 4123 // This blocks doesn't need any recovery. 4124 // We can remove this block and close the file. 4125 pendingFile.removeLastBlock(lastBlock); 4126 finalizeINodeFileUnderConstruction(src, pendingFile, 4127 iip.getLatestSnapshotId()); 4128 NameNode.stateChangeLog.warn("BLOCK* internalReleaseLease: " 4129 + "Removed empty last block and closed file."); 4130 return true; 4131 } 4132 // start recovery of the last block for this file 4133 long blockRecoveryId = nextGenerationStamp(blockIdManager.isLegacyBlock(uc)); 4134 lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile); 4135 if(copyOnTruncate) { 4136 uc.setGenerationStamp(blockRecoveryId); 4137 } else if(truncateRecovery) { 4138 recoveryBlock.setGenerationStamp(blockRecoveryId); 4139 } 4140 uc.initializeBlockRecovery(blockRecoveryId); 4141 leaseManager.renewLease(lease); 4142 // Cannot close file right now, since the last block requires recovery. 4143 // This may potentially cause infinite loop in lease recovery 4144 // if there are no valid replicas on data-nodes. 4145 NameNode.stateChangeLog.warn( 4146 "DIR* NameSystem.internalReleaseLease: " + 4147 "File " + src + " has not been closed." + 4148 " Lease recovery is in progress. " + 4149 "RecoveryId = " + blockRecoveryId + " for block " + lastBlock); 4150 break; 4151 } 4152 return false; 4153 } 4154 4155 private Lease reassignLease(Lease lease, String src, String newHolder, 4156 INodeFile pendingFile) { 4157 assert hasWriteLock(); 4158 if(newHolder == null) 4159 return lease; 4160 // The following transaction is not synced. Make sure it's sync'ed later. 4161 logReassignLease(lease.getHolder(), src, newHolder); 4162 return reassignLeaseInternal(lease, src, newHolder, pendingFile); 4163 } 4164 4165 Lease reassignLeaseInternal(Lease lease, String src, String newHolder, 4166 INodeFile pendingFile) { 4167 assert hasWriteLock(); 4168 pendingFile.getFileUnderConstructionFeature().setClientName(newHolder); 4169 return leaseManager.reassignLease(lease, src, newHolder); 4170 } 4171 4172 private void commitOrCompleteLastBlock(final INodeFile fileINode, 4173 final INodesInPath iip, final Block commitBlock) throws IOException { 4174 assert hasWriteLock(); 4175 Preconditions.checkArgument(fileINode.isUnderConstruction()); 4176 if (!blockManager.commitOrCompleteLastBlock(fileINode, commitBlock)) { 4177 return; 4178 } 4179 4180 // Adjust disk space consumption if required 4181 final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes(); 4182 if (diff > 0) { 4183 try { 4184 dir.updateSpaceConsumed(iip, 0, -diff, fileINode.getFileReplication()); 4185 } catch (IOException e) { 4186 LOG.warn("Unexpected exception while updating disk space.", e); 4187 } 4188 } 4189 } 4190 4191 private void finalizeINodeFileUnderConstruction(String src, 4192 INodeFile pendingFile, int latestSnapshot) throws IOException { 4193 assert hasWriteLock(); 4194 4195 FileUnderConstructionFeature uc = pendingFile.getFileUnderConstructionFeature(); 4196 if (uc == null) { 4197 throw new IOException("Cannot finalize file " + src 4198 + " because it is not under construction"); 4199 } 4200 leaseManager.removeLease(uc.getClientName(), src); 4201 4202 pendingFile.recordModification(latestSnapshot); 4203 4204 // The file is no longer pending. 4205 // Create permanent INode, update blocks. No need to replace the inode here 4206 // since we just remove the uc feature from pendingFile 4207 pendingFile.toCompleteFile(now()); 4208 4209 waitForLoadingFSImage(); 4210 // close file and persist block allocations for this file 4211 closeFile(src, pendingFile); 4212 4213 blockManager.checkReplication(pendingFile); 4214 } 4215 4216 @VisibleForTesting 4217 BlockInfoContiguous getStoredBlock(Block block) { 4218 return blockManager.getStoredBlock(block); 4219 } 4220 4221 @Override 4222 public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC) { 4223 assert hasReadLock(); 4224 final BlockCollection bc = blockUC.getBlockCollection(); 4225 if (bc == null || !(bc instanceof INodeFile) 4226 || !bc.isUnderConstruction()) { 4227 return false; 4228 } 4229 4230 String fullName = bc.getName(); 4231 try { 4232 if (fullName != null && fullName.startsWith(Path.SEPARATOR) 4233 && dir.getINode(fullName) == bc) { 4234 // If file exists in normal path then no need to look in snapshot 4235 return false; 4236 } 4237 } catch (UnresolvedLinkException e) { 4238 LOG.error("Error while resolving the link : " + fullName, e); 4239 return false; 4240 } 4241 /* 4242 * 1. if bc is under construction and also with snapshot, and 4243 * bc is not in the current fsdirectory tree, bc must represent a snapshot 4244 * file. 4245 * 2. if fullName is not an absolute path, bc cannot be existent in the 4246 * current fsdirectory tree. 4247 * 3. if bc is not the current node associated with fullName, bc must be a 4248 * snapshot inode. 4249 */ 4250 return true; 4251 } 4252 4253 void commitBlockSynchronization(ExtendedBlock oldBlock, 4254 long newgenerationstamp, long newlength, 4255 boolean closeFile, boolean deleteblock, DatanodeID[] newtargets, 4256 String[] newtargetstorages) throws IOException { 4257 LOG.info("commitBlockSynchronization(oldBlock=" + oldBlock 4258 + ", newgenerationstamp=" + newgenerationstamp 4259 + ", newlength=" + newlength 4260 + ", newtargets=" + Arrays.asList(newtargets) 4261 + ", closeFile=" + closeFile 4262 + ", deleteBlock=" + deleteblock 4263 + ")"); 4264 checkOperation(OperationCategory.WRITE); 4265 final String src; 4266 waitForLoadingFSImage(); 4267 writeLock(); 4268 try { 4269 checkOperation(OperationCategory.WRITE); 4270 // If a DN tries to commit to the standby, the recovery will 4271 // fail, and the next retry will succeed on the new NN. 4272 4273 checkNameNodeSafeMode( 4274 "Cannot commitBlockSynchronization while in safe mode"); 4275 final BlockInfoContiguous storedBlock = getStoredBlock( 4276 ExtendedBlock.getLocalBlock(oldBlock)); 4277 if (storedBlock == null) { 4278 if (deleteblock) { 4279 // This may be a retry attempt so ignore the failure 4280 // to locate the block. 4281 if (LOG.isDebugEnabled()) { 4282 LOG.debug("Block (=" + oldBlock + ") not found"); 4283 } 4284 return; 4285 } else { 4286 throw new IOException("Block (=" + oldBlock + ") not found"); 4287 } 4288 } 4289 final long oldGenerationStamp = storedBlock.getGenerationStamp(); 4290 final long oldNumBytes = storedBlock.getNumBytes(); 4291 // 4292 // The implementation of delete operation (see @deleteInternal method) 4293 // first removes the file paths from namespace, and delays the removal 4294 // of blocks to later time for better performance. When 4295 // commitBlockSynchronization (this method) is called in between, the 4296 // blockCollection of storedBlock could have been assigned to null by 4297 // the delete operation, throw IOException here instead of NPE; if the 4298 // file path is already removed from namespace by the delete operation, 4299 // throw FileNotFoundException here, so not to proceed to the end of 4300 // this method to add a CloseOp to the edit log for an already deleted 4301 // file (See HDFS-6825). 4302 // 4303 BlockCollection blockCollection = storedBlock.getBlockCollection(); 4304 if (blockCollection == null) { 4305 throw new IOException("The blockCollection of " + storedBlock 4306 + " is null, likely because the file owning this block was" 4307 + " deleted and the block removal is delayed"); 4308 } 4309 INodeFile iFile = ((INode)blockCollection).asFile(); 4310 src = iFile.getFullPathName(); 4311 if (isFileDeleted(iFile)) { 4312 throw new FileNotFoundException("File not found: " 4313 + src + ", likely due to delayed block removal"); 4314 } 4315 if ((!iFile.isUnderConstruction() || storedBlock.isComplete()) && 4316 iFile.getLastBlock().isComplete()) { 4317 if (LOG.isDebugEnabled()) { 4318 LOG.debug("Unexpected block (=" + oldBlock 4319 + ") since the file (=" + iFile.getLocalName() 4320 + ") is not under construction"); 4321 } 4322 return; 4323 } 4324 4325 BlockInfoContiguousUnderConstruction truncatedBlock = 4326 (BlockInfoContiguousUnderConstruction) iFile.getLastBlock(); 4327 long recoveryId = truncatedBlock.getBlockRecoveryId(); 4328 boolean copyTruncate = 4329 truncatedBlock.getBlockId() != storedBlock.getBlockId(); 4330 if(recoveryId != newgenerationstamp) { 4331 throw new IOException("The recovery id " + newgenerationstamp 4332 + " does not match current recovery id " 4333 + recoveryId + " for block " + oldBlock); 4334 } 4335 4336 if (deleteblock) { 4337 Block blockToDel = ExtendedBlock.getLocalBlock(oldBlock); 4338 boolean remove = iFile.removeLastBlock(blockToDel); 4339 if (remove) { 4340 blockManager.removeBlock(storedBlock); 4341 } 4342 } 4343 else { 4344 // update last block 4345 if(!copyTruncate) { 4346 storedBlock.setGenerationStamp(newgenerationstamp); 4347 storedBlock.setNumBytes(newlength); 4348 } 4349 4350 // find the DatanodeDescriptor objects 4351 ArrayList<DatanodeDescriptor> trimmedTargets = 4352 new ArrayList<DatanodeDescriptor>(newtargets.length); 4353 ArrayList<String> trimmedStorages = 4354 new ArrayList<String>(newtargets.length); 4355 if (newtargets.length > 0) { 4356 for (int i = 0; i < newtargets.length; ++i) { 4357 // try to get targetNode 4358 DatanodeDescriptor targetNode = 4359 blockManager.getDatanodeManager().getDatanode(newtargets[i]); 4360 if (targetNode != null) { 4361 trimmedTargets.add(targetNode); 4362 trimmedStorages.add(newtargetstorages[i]); 4363 } else if (LOG.isDebugEnabled()) { 4364 LOG.debug("DatanodeDescriptor (=" + newtargets[i] + ") not found"); 4365 } 4366 } 4367 } 4368 if ((closeFile) && !trimmedTargets.isEmpty()) { 4369 // the file is getting closed. Insert block locations into blockManager. 4370 // Otherwise fsck will report these blocks as MISSING, especially if the 4371 // blocksReceived from Datanodes take a long time to arrive. 4372 for (int i = 0; i < trimmedTargets.size(); i++) { 4373 DatanodeStorageInfo storageInfo = 4374 trimmedTargets.get(i).getStorageInfo(trimmedStorages.get(i)); 4375 if (storageInfo != null) { 4376 if(copyTruncate) { 4377 storageInfo.addBlock(truncatedBlock); 4378 } else { 4379 storageInfo.addBlock(storedBlock); 4380 } 4381 } 4382 } 4383 } 4384 4385 // add pipeline locations into the INodeUnderConstruction 4386 DatanodeStorageInfo[] trimmedStorageInfos = 4387 blockManager.getDatanodeManager().getDatanodeStorageInfos( 4388 trimmedTargets.toArray(new DatanodeID[trimmedTargets.size()]), 4389 trimmedStorages.toArray(new String[trimmedStorages.size()]), 4390 "src=%s, oldBlock=%s, newgenerationstamp=%d, newlength=%d", 4391 src, oldBlock, newgenerationstamp, newlength); 4392 4393 if(copyTruncate) { 4394 iFile.setLastBlock(truncatedBlock, trimmedStorageInfos); 4395 } else { 4396 iFile.setLastBlock(storedBlock, trimmedStorageInfos); 4397 if (closeFile) { 4398 blockManager.markBlockReplicasAsCorrupt(storedBlock, 4399 oldGenerationStamp, oldNumBytes, trimmedStorageInfos); 4400 } 4401 } 4402 } 4403 4404 if (closeFile) { 4405 if(copyTruncate) { 4406 closeFileCommitBlocks(src, iFile, truncatedBlock); 4407 if(!iFile.isBlockInLatestSnapshot(storedBlock)) { 4408 blockManager.removeBlock(storedBlock); 4409 } 4410 } else { 4411 closeFileCommitBlocks(src, iFile, storedBlock); 4412 } 4413 } else { 4414 // If this commit does not want to close the file, persist blocks 4415 persistBlocks(src, iFile, false); 4416 } 4417 } finally { 4418 writeUnlock(); 4419 } 4420 getEditLog().logSync(); 4421 if (closeFile) { 4422 LOG.info("commitBlockSynchronization(oldBlock=" + oldBlock 4423 + ", file=" + src 4424 + ", newgenerationstamp=" + newgenerationstamp 4425 + ", newlength=" + newlength 4426 + ", newtargets=" + Arrays.asList(newtargets) + ") successful"); 4427 } else { 4428 LOG.info("commitBlockSynchronization(" + oldBlock + ") successful"); 4429 } 4430 } 4431 4432 /** 4433 * @param pendingFile open file that needs to be closed 4434 * @param storedBlock last block 4435 * @return Path of the file that was closed. 4436 * @throws IOException on error 4437 */ 4438 @VisibleForTesting 4439 void closeFileCommitBlocks(String src, INodeFile pendingFile, 4440 BlockInfoContiguous storedBlock) throws IOException { 4441 final INodesInPath iip = INodesInPath.fromINode(pendingFile); 4442 4443 // commit the last block and complete it if it has minimum replicas 4444 commitOrCompleteLastBlock(pendingFile, iip, storedBlock); 4445 4446 //remove lease, close file 4447 finalizeINodeFileUnderConstruction(src, pendingFile, 4448 Snapshot.findLatestSnapshot(pendingFile, Snapshot.CURRENT_STATE_ID)); 4449 } 4450 4451 /** 4452 * Renew the lease(s) held by the given client 4453 */ 4454 void renewLease(String holder) throws IOException { 4455 checkOperation(OperationCategory.WRITE); 4456 readLock(); 4457 try { 4458 checkOperation(OperationCategory.WRITE); 4459 checkNameNodeSafeMode("Cannot renew lease for " + holder); 4460 leaseManager.renewLease(holder); 4461 } finally { 4462 readUnlock(); 4463 } 4464 } 4465 4466 /** 4467 * Get a partial listing of the indicated directory 4468 * 4469 * @param src the directory name 4470 * @param startAfter the name to start after 4471 * @param needLocation if blockLocations need to be returned 4472 * @return a partial listing starting after startAfter 4473 * 4474 * @throws AccessControlException if access is denied 4475 * @throws UnresolvedLinkException if symbolic link is encountered 4476 * @throws IOException if other I/O error occurred 4477 */ 4478 DirectoryListing getListing(String src, byte[] startAfter, 4479 boolean needLocation) 4480 throws IOException { 4481 checkOperation(OperationCategory.READ); 4482 DirectoryListing dl = null; 4483 readLock(); 4484 try { 4485 checkOperation(NameNode.OperationCategory.READ); 4486 dl = FSDirStatAndListingOp.getListingInt(dir, src, startAfter, 4487 needLocation); 4488 } catch (AccessControlException e) { 4489 logAuditEvent(false, "listStatus", src); 4490 throw e; 4491 } finally { 4492 readUnlock(); 4493 } 4494 logAuditEvent(true, "listStatus", src); 4495 return dl; 4496 } 4497 4498 ///////////////////////////////////////////////////////// 4499 // 4500 // These methods are called by datanodes 4501 // 4502 ///////////////////////////////////////////////////////// 4503 /** 4504 * Register Datanode. 4505 * <p> 4506 * The purpose of registration is to identify whether the new datanode 4507 * serves a new data storage, and will report new data block copies, 4508 * which the namenode was not aware of; or the datanode is a replacement 4509 * node for the data storage that was previously served by a different 4510 * or the same (in terms of host:port) datanode. 4511 * The data storages are distinguished by their storageIDs. When a new 4512 * data storage is reported the namenode issues a new unique storageID. 4513 * <p> 4514 * Finally, the namenode returns its namespaceID as the registrationID 4515 * for the datanodes. 4516 * namespaceID is a persistent attribute of the name space. 4517 * The registrationID is checked every time the datanode is communicating 4518 * with the namenode. 4519 * Datanodes with inappropriate registrationID are rejected. 4520 * If the namenode stops, and then restarts it can restore its 4521 * namespaceID and will continue serving the datanodes that has previously 4522 * registered with the namenode without restarting the whole cluster. 4523 * 4524 * @see org.apache.hadoop.hdfs.server.datanode.DataNode 4525 */ 4526 void registerDatanode(DatanodeRegistration nodeReg) throws IOException { 4527 writeLock(); 4528 try { 4529 getBlockManager().getDatanodeManager().registerDatanode(nodeReg); 4530 checkSafeMode(); 4531 } finally { 4532 writeUnlock(); 4533 } 4534 } 4535 4536 /** 4537 * Get registrationID for datanodes based on the namespaceID. 4538 * 4539 * @see #registerDatanode(DatanodeRegistration) 4540 * @return registration ID 4541 */ 4542 String getRegistrationID() { 4543 return Storage.getRegistrationID(getFSImage().getStorage()); 4544 } 4545 4546 /** 4547 * The given node has reported in. This method should: 4548 * 1) Record the heartbeat, so the datanode isn't timed out 4549 * 2) Adjust usage stats for future block allocation 4550 * 4551 * If a substantial amount of time passed since the last datanode 4552 * heartbeat then request an immediate block report. 4553 * 4554 * @return an array of datanode commands 4555 * @throws IOException 4556 */ 4557 HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg, 4558 StorageReport[] reports, long cacheCapacity, long cacheUsed, 4559 int xceiverCount, int xmitsInProgress, int failedVolumes, 4560 VolumeFailureSummary volumeFailureSummary) throws IOException { 4561 readLock(); 4562 try { 4563 //get datanode commands 4564 final int maxTransfer = blockManager.getMaxReplicationStreams() 4565 - xmitsInProgress; 4566 DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat( 4567 nodeReg, reports, blockPoolId, cacheCapacity, cacheUsed, 4568 xceiverCount, maxTransfer, failedVolumes, volumeFailureSummary); 4569 4570 //create ha status 4571 final NNHAStatusHeartbeat haState = new NNHAStatusHeartbeat( 4572 haContext.getState().getServiceState(), 4573 getFSImage().getLastAppliedOrWrittenTxId()); 4574 4575 return new HeartbeatResponse(cmds, haState, rollingUpgradeInfo); 4576 } finally { 4577 readUnlock(); 4578 } 4579 } 4580 4581 /** 4582 * Returns whether or not there were available resources at the last check of 4583 * resources. 4584 * 4585 * @return true if there were sufficient resources available, false otherwise. 4586 */ 4587 boolean nameNodeHasResourcesAvailable() { 4588 return hasResourcesAvailable; 4589 } 4590 4591 /** 4592 * Perform resource checks and cache the results. 4593 */ 4594 void checkAvailableResources() { 4595 Preconditions.checkState(nnResourceChecker != null, 4596 "nnResourceChecker not initialized"); 4597 hasResourcesAvailable = nnResourceChecker.hasAvailableDiskSpace(); 4598 } 4599 4600 /** 4601 * Persist the block list for the inode. 4602 * @param path 4603 * @param file 4604 * @param logRetryCache 4605 */ 4606 private void persistBlocks(String path, INodeFile file, 4607 boolean logRetryCache) { 4608 assert hasWriteLock(); 4609 Preconditions.checkArgument(file.isUnderConstruction()); 4610 getEditLog().logUpdateBlocks(path, file, logRetryCache); 4611 NameNode.stateChangeLog.debug("persistBlocks: {} with {} blocks is" + 4612 " peristed to the file system", path, file.getBlocks().length); 4613 } 4614 4615 /** 4616 * Close file. 4617 * @param path 4618 * @param file 4619 */ 4620 private void closeFile(String path, INodeFile file) { 4621 assert hasWriteLock(); 4622 waitForLoadingFSImage(); 4623 // file is closed 4624 getEditLog().logCloseFile(path, file); 4625 NameNode.stateChangeLog.debug("closeFile: {} with {} blocks is persisted" + 4626 " to the file system", path, file.getBlocks().length); 4627 } 4628 4629 /** 4630 * Periodically calls hasAvailableResources of NameNodeResourceChecker, and if 4631 * there are found to be insufficient resources available, causes the NN to 4632 * enter safe mode. If resources are later found to have returned to 4633 * acceptable levels, this daemon will cause the NN to exit safe mode. 4634 */ 4635 class NameNodeResourceMonitor implements Runnable { 4636 boolean shouldNNRmRun = true; 4637 @Override 4638 public void run () { 4639 try { 4640 while (fsRunning && shouldNNRmRun) { 4641 checkAvailableResources(); 4642 if(!nameNodeHasResourcesAvailable()) { 4643 String lowResourcesMsg = "NameNode low on available disk space. "; 4644 if (!isInSafeMode()) { 4645 LOG.warn(lowResourcesMsg + "Entering safe mode."); 4646 } else { 4647 LOG.warn(lowResourcesMsg + "Already in safe mode."); 4648 } 4649 enterSafeMode(true); 4650 } 4651 try { 4652 Thread.sleep(resourceRecheckInterval); 4653 } catch (InterruptedException ie) { 4654 // Deliberately ignore 4655 } 4656 } 4657 } catch (Exception e) { 4658 FSNamesystem.LOG.error("Exception in NameNodeResourceMonitor: ", e); 4659 } 4660 } 4661 4662 public void stopMonitor() { 4663 shouldNNRmRun = false; 4664 } 4665 } 4666 4667 class NameNodeEditLogRoller implements Runnable { 4668 4669 private boolean shouldRun = true; 4670 private final long rollThreshold; 4671 private final long sleepIntervalMs; 4672 4673 public NameNodeEditLogRoller(long rollThreshold, int sleepIntervalMs) { 4674 this.rollThreshold = rollThreshold; 4675 this.sleepIntervalMs = sleepIntervalMs; 4676 } 4677 4678 @Override 4679 public void run() { 4680 while (fsRunning && shouldRun) { 4681 try { 4682 FSEditLog editLog = getFSImage().getEditLog(); 4683 long numEdits = 4684 editLog.getLastWrittenTxId() - editLog.getCurSegmentTxId(); 4685 if (numEdits > rollThreshold) { 4686 FSNamesystem.LOG.info("NameNode rolling its own edit log because" 4687 + " number of edits in open segment exceeds threshold of " 4688 + rollThreshold); 4689 rollEditLog(); 4690 } 4691 } catch (Exception e) { 4692 FSNamesystem.LOG.error("Swallowing exception in " 4693 + NameNodeEditLogRoller.class.getSimpleName() + ":", e); 4694 } 4695 try { 4696 Thread.sleep(sleepIntervalMs); 4697 } catch (InterruptedException e) { 4698 FSNamesystem.LOG.info(NameNodeEditLogRoller.class.getSimpleName() 4699 + " was interrupted, exiting"); 4700 break; 4701 } 4702 } 4703 } 4704 4705 public void stop() { 4706 shouldRun = false; 4707 } 4708 } 4709 4710 /** 4711 * Daemon to periodically scan the namespace for lazyPersist files 4712 * with missing blocks and unlink them. 4713 */ 4714 class LazyPersistFileScrubber implements Runnable { 4715 private volatile boolean shouldRun = true; 4716 final int scrubIntervalSec; 4717 public LazyPersistFileScrubber(final int scrubIntervalSec) { 4718 this.scrubIntervalSec = scrubIntervalSec; 4719 } 4720 4721 /** 4722 * Periodically go over the list of lazyPersist files with missing 4723 * blocks and unlink them from the namespace. 4724 */ 4725 private void clearCorruptLazyPersistFiles() 4726 throws IOException { 4727 4728 BlockStoragePolicy lpPolicy = blockManager.getStoragePolicy("LAZY_PERSIST"); 4729 4730 List<BlockCollection> filesToDelete = new ArrayList<>(); 4731 boolean changed = false; 4732 writeLock(); 4733 try { 4734 final Iterator<Block> it = blockManager.getCorruptReplicaBlockIterator(); 4735 4736 while (it.hasNext()) { 4737 Block b = it.next(); 4738 BlockInfoContiguous blockInfo = blockManager.getStoredBlock(b); 4739 if (blockInfo.getBlockCollection().getStoragePolicyID() 4740 == lpPolicy.getId()) { 4741 filesToDelete.add(blockInfo.getBlockCollection()); 4742 } 4743 } 4744 4745 for (BlockCollection bc : filesToDelete) { 4746 LOG.warn("Removing lazyPersist file " + bc.getName() + " with no replicas."); 4747 BlocksMapUpdateInfo toRemoveBlocks = 4748 FSDirDeleteOp.deleteInternal( 4749 FSNamesystem.this, bc.getName(), 4750 INodesInPath.fromINode((INodeFile) bc), false); 4751 changed |= toRemoveBlocks != null; 4752 if (toRemoveBlocks != null) { 4753 removeBlocks(toRemoveBlocks); // Incremental deletion of blocks 4754 } 4755 } 4756 } finally { 4757 writeUnlock(); 4758 } 4759 if (changed) { 4760 getEditLog().logSync(); 4761 } 4762 } 4763 4764 @Override 4765 public void run() { 4766 while (fsRunning && shouldRun) { 4767 try { 4768 clearCorruptLazyPersistFiles(); 4769 } catch (Exception e) { 4770 FSNamesystem.LOG.error( 4771 "Ignoring exception in LazyPersistFileScrubber:", e); 4772 } 4773 4774 try { 4775 Thread.sleep(scrubIntervalSec * 1000); 4776 } catch (InterruptedException e) { 4777 FSNamesystem.LOG.info( 4778 "LazyPersistFileScrubber was interrupted, exiting"); 4779 break; 4780 } 4781 } 4782 } 4783 4784 public void stop() { 4785 shouldRun = false; 4786 } 4787 } 4788 4789 public FSImage getFSImage() { 4790 return fsImage; 4791 } 4792 4793 public FSEditLog getEditLog() { 4794 return getFSImage().getEditLog(); 4795 } 4796 4797 private void checkBlock(ExtendedBlock block) throws IOException { 4798 if (block != null && !this.blockPoolId.equals(block.getBlockPoolId())) { 4799 throw new IOException("Unexpected BlockPoolId " + block.getBlockPoolId() 4800 + " - expected " + blockPoolId); 4801 } 4802 } 4803 4804 @Metric({"MissingBlocks", "Number of missing blocks"}) 4805 public long getMissingBlocksCount() { 4806 // not locking 4807 return blockManager.getMissingBlocksCount(); 4808 } 4809 4810 @Metric({"MissingReplOneBlocks", "Number of missing blocks " + 4811 "with replication factor 1"}) 4812 public long getMissingReplOneBlocksCount() { 4813 // not locking 4814 return blockManager.getMissingReplOneBlocksCount(); 4815 } 4816 4817 @Metric({"ExpiredHeartbeats", "Number of expired heartbeats"}) 4818 public int getExpiredHeartbeats() { 4819 return datanodeStatistics.getExpiredHeartbeats(); 4820 } 4821 4822 @Metric({"TransactionsSinceLastCheckpoint", 4823 "Number of transactions since last checkpoint"}) 4824 public long getTransactionsSinceLastCheckpoint() { 4825 return getEditLog().getLastWrittenTxId() - 4826 getFSImage().getStorage().getMostRecentCheckpointTxId(); 4827 } 4828 4829 @Metric({"TransactionsSinceLastLogRoll", 4830 "Number of transactions since last edit log roll"}) 4831 public long getTransactionsSinceLastLogRoll() { 4832 if (isInStandbyState() || !getEditLog().isSegmentOpen()) { 4833 return 0; 4834 } else { 4835 return getEditLog().getLastWrittenTxId() - 4836 getEditLog().getCurSegmentTxId() + 1; 4837 } 4838 } 4839 4840 @Metric({"LastWrittenTransactionId", "Transaction ID written to the edit log"}) 4841 public long getLastWrittenTransactionId() { 4842 return getEditLog().getLastWrittenTxId(); 4843 } 4844 4845 @Metric({"LastCheckpointTime", 4846 "Time in milliseconds since the epoch of the last checkpoint"}) 4847 public long getLastCheckpointTime() { 4848 return getFSImage().getStorage().getMostRecentCheckpointTime(); 4849 } 4850 4851 /** @see ClientProtocol#getStats() */ 4852 long[] getStats() { 4853 final long[] stats = datanodeStatistics.getStats(); 4854 stats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = getUnderReplicatedBlocks(); 4855 stats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = getCorruptReplicaBlocks(); 4856 stats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = getMissingBlocksCount(); 4857 stats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] = 4858 getMissingReplOneBlocksCount(); 4859 return stats; 4860 } 4861 4862 @Override // FSNamesystemMBean 4863 @Metric({"CapacityTotal", 4864 "Total raw capacity of data nodes in bytes"}) 4865 public long getCapacityTotal() { 4866 return datanodeStatistics.getCapacityTotal(); 4867 } 4868 4869 @Metric({"CapacityTotalGB", 4870 "Total raw capacity of data nodes in GB"}) 4871 public float getCapacityTotalGB() { 4872 return DFSUtil.roundBytesToGB(getCapacityTotal()); 4873 } 4874 4875 @Override // FSNamesystemMBean 4876 @Metric({"CapacityUsed", 4877 "Total used capacity across all data nodes in bytes"}) 4878 public long getCapacityUsed() { 4879 return datanodeStatistics.getCapacityUsed(); 4880 } 4881 4882 @Metric({"CapacityUsedGB", 4883 "Total used capacity across all data nodes in GB"}) 4884 public float getCapacityUsedGB() { 4885 return DFSUtil.roundBytesToGB(getCapacityUsed()); 4886 } 4887 4888 @Override // FSNamesystemMBean 4889 @Metric({"CapacityRemaining", "Remaining capacity in bytes"}) 4890 public long getCapacityRemaining() { 4891 return datanodeStatistics.getCapacityRemaining(); 4892 } 4893 4894 @Metric({"CapacityRemainingGB", "Remaining capacity in GB"}) 4895 public float getCapacityRemainingGB() { 4896 return DFSUtil.roundBytesToGB(getCapacityRemaining()); 4897 } 4898 4899 @Metric({"CapacityUsedNonDFS", 4900 "Total space used by data nodes for non DFS purposes in bytes"}) 4901 public long getCapacityUsedNonDFS() { 4902 return datanodeStatistics.getCapacityUsedNonDFS(); 4903 } 4904 4905 /** 4906 * Total number of connections. 4907 */ 4908 @Override // FSNamesystemMBean 4909 @Metric 4910 public int getTotalLoad() { 4911 return datanodeStatistics.getXceiverCount(); 4912 } 4913 4914 @Metric({ "SnapshottableDirectories", "Number of snapshottable directories" }) 4915 public int getNumSnapshottableDirs() { 4916 return this.snapshotManager.getNumSnapshottableDirs(); 4917 } 4918 4919 @Metric({ "Snapshots", "The number of snapshots" }) 4920 public int getNumSnapshots() { 4921 return this.snapshotManager.getNumSnapshots(); 4922 } 4923 4924 @Override 4925 public String getSnapshotStats() { 4926 Map<String, Object> info = new HashMap<String, Object>(); 4927 info.put("SnapshottableDirectories", this.getNumSnapshottableDirs()); 4928 info.put("Snapshots", this.getNumSnapshots()); 4929 return JSON.toString(info); 4930 } 4931 4932 int getNumberOfDatanodes(DatanodeReportType type) { 4933 readLock(); 4934 try { 4935 return getBlockManager().getDatanodeManager().getDatanodeListForReport( 4936 type).size(); 4937 } finally { 4938 readUnlock(); 4939 } 4940 } 4941 4942 DatanodeInfo[] datanodeReport(final DatanodeReportType type 4943 ) throws AccessControlException, StandbyException { 4944 checkSuperuserPrivilege(); 4945 checkOperation(OperationCategory.UNCHECKED); 4946 readLock(); 4947 try { 4948 checkOperation(OperationCategory.UNCHECKED); 4949 final DatanodeManager dm = getBlockManager().getDatanodeManager(); 4950 final List<DatanodeDescriptor> results = dm.getDatanodeListForReport(type); 4951 4952 DatanodeInfo[] arr = new DatanodeInfo[results.size()]; 4953 for (int i=0; i<arr.length; i++) { 4954 arr[i] = new DatanodeInfo(results.get(i)); 4955 } 4956 return arr; 4957 } finally { 4958 readUnlock(); 4959 } 4960 } 4961 4962 DatanodeStorageReport[] getDatanodeStorageReport(final DatanodeReportType type 4963 ) throws AccessControlException, StandbyException { 4964 checkSuperuserPrivilege(); 4965 checkOperation(OperationCategory.UNCHECKED); 4966 readLock(); 4967 try { 4968 checkOperation(OperationCategory.UNCHECKED); 4969 final DatanodeManager dm = getBlockManager().getDatanodeManager(); 4970 final List<DatanodeDescriptor> datanodes = dm.getDatanodeListForReport(type); 4971 4972 DatanodeStorageReport[] reports = new DatanodeStorageReport[datanodes.size()]; 4973 for (int i = 0; i < reports.length; i++) { 4974 final DatanodeDescriptor d = datanodes.get(i); 4975 reports[i] = new DatanodeStorageReport(new DatanodeInfo(d), 4976 d.getStorageReports()); 4977 } 4978 return reports; 4979 } finally { 4980 readUnlock(); 4981 } 4982 } 4983 4984 /** 4985 * Save namespace image. 4986 * This will save current namespace into fsimage file and empty edits file. 4987 * Requires superuser privilege and safe mode. 4988 * 4989 * @throws AccessControlException if superuser privilege is violated. 4990 * @throws IOException if 4991 */ 4992 void saveNamespace() throws AccessControlException, IOException { 4993 checkOperation(OperationCategory.UNCHECKED); 4994 checkSuperuserPrivilege(); 4995 4996 cpLock(); // Block if a checkpointing is in progress on standby. 4997 readLock(); 4998 try { 4999 checkOperation(OperationCategory.UNCHECKED); 5000 5001 if (!isInSafeMode()) { 5002 throw new IOException("Safe mode should be turned ON " 5003 + "in order to create namespace image."); 5004 } 5005 getFSImage().saveNamespace(this); 5006 } finally { 5007 readUnlock(); 5008 cpUnlock(); 5009 } 5010 LOG.info("New namespace image has been created"); 5011 } 5012 5013 /** 5014 * Enables/Disables/Checks restoring failed storage replicas if the storage becomes available again. 5015 * Requires superuser privilege. 5016 * 5017 * @throws AccessControlException if superuser privilege is violated. 5018 */ 5019 boolean restoreFailedStorage(String arg) throws AccessControlException, 5020 StandbyException { 5021 checkSuperuserPrivilege(); 5022 checkOperation(OperationCategory.UNCHECKED); 5023 cpLock(); // Block if a checkpointing is in progress on standby. 5024 writeLock(); 5025 try { 5026 checkOperation(OperationCategory.UNCHECKED); 5027 5028 // if it is disabled - enable it and vice versa. 5029 if(arg.equals("check")) 5030 return getFSImage().getStorage().getRestoreFailedStorage(); 5031 5032 boolean val = arg.equals("true"); // false if not 5033 getFSImage().getStorage().setRestoreFailedStorage(val); 5034 5035 return val; 5036 } finally { 5037 writeUnlock(); 5038 cpUnlock(); 5039 } 5040 } 5041 5042 Date getStartTime() { 5043 return new Date(startTime); 5044 } 5045 5046 void finalizeUpgrade() throws IOException { 5047 checkSuperuserPrivilege(); 5048 checkOperation(OperationCategory.UNCHECKED); 5049 cpLock(); // Block if a checkpointing is in progress on standby. 5050 writeLock(); 5051 try { 5052 checkOperation(OperationCategory.UNCHECKED); 5053 getFSImage().finalizeUpgrade(this.isHaEnabled() && inActiveState()); 5054 } finally { 5055 writeUnlock(); 5056 cpUnlock(); 5057 } 5058 } 5059 5060 void refreshNodes() throws IOException { 5061 checkOperation(OperationCategory.UNCHECKED); 5062 checkSuperuserPrivilege(); 5063 getBlockManager().getDatanodeManager().refreshNodes(new HdfsConfiguration()); 5064 } 5065 5066 void setBalancerBandwidth(long bandwidth) throws IOException { 5067 checkOperation(OperationCategory.UNCHECKED); 5068 checkSuperuserPrivilege(); 5069 getBlockManager().getDatanodeManager().setBalancerBandwidth(bandwidth); 5070 } 5071 5072 /** 5073 * Persist the new block (the last block of the given file). 5074 * @param path 5075 * @param file 5076 */ 5077 private void persistNewBlock(String path, INodeFile file) { 5078 Preconditions.checkArgument(file.isUnderConstruction()); 5079 getEditLog().logAddBlock(path, file); 5080 NameNode.stateChangeLog.debug("persistNewBlock: {} with new block {}," + 5081 " current total block count is {}", path, 5082 file.getLastBlock().toString(), file.getBlocks().length); 5083 } 5084 5085 /** 5086 * SafeModeInfo contains information related to the safe mode. 5087 * <p> 5088 * An instance of {@link SafeModeInfo} is created when the name node 5089 * enters safe mode. 5090 * <p> 5091 * During name node startup {@link SafeModeInfo} counts the number of 5092 * <em>safe blocks</em>, those that have at least the minimal number of 5093 * replicas, and calculates the ratio of safe blocks to the total number 5094 * of blocks in the system, which is the size of blocks in 5095 * {@link FSNamesystem#blockManager}. When the ratio reaches the 5096 * {@link #threshold} it starts the SafeModeMonitor daemon in order 5097 * to monitor whether the safe mode {@link #extension} is passed. 5098 * Then it leaves safe mode and destroys itself. 5099 * <p> 5100 * If safe mode is turned on manually then the number of safe blocks is 5101 * not tracked because the name node is not intended to leave safe mode 5102 * automatically in the case. 5103 * 5104 * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction, boolean) 5105 */ 5106 public class SafeModeInfo { 5107 // configuration fields 5108 /** Safe mode threshold condition %.*/ 5109 private final double threshold; 5110 /** Safe mode minimum number of datanodes alive */ 5111 private final int datanodeThreshold; 5112 /** 5113 * Safe mode extension after the threshold. 5114 * Make it volatile so that getSafeModeTip can read the latest value 5115 * without taking a lock. 5116 */ 5117 private volatile int extension; 5118 /** Min replication required by safe mode. */ 5119 private final int safeReplication; 5120 /** threshold for populating needed replication queues */ 5121 private final double replQueueThreshold; 5122 // internal fields 5123 /** Time when threshold was reached. 5124 * <br> -1 safe mode is off 5125 * <br> 0 safe mode is on, and threshold is not reached yet 5126 * <br> >0 safe mode is on, but we are in extension period 5127 */ 5128 private long reached = -1; 5129 private long reachedTimestamp = -1; 5130 /** Total number of blocks. */ 5131 int blockTotal; 5132 /** Number of safe blocks. */ 5133 int blockSafe; 5134 /** Number of blocks needed to satisfy safe mode threshold condition */ 5135 private int blockThreshold; 5136 /** Number of blocks needed before populating replication queues */ 5137 private int blockReplQueueThreshold; 5138 /** time of the last status printout */ 5139 private long lastStatusReport = 0; 5140 /** 5141 * Was safemode entered automatically because available resources were low. 5142 * Make it volatile so that getSafeModeTip can read the latest value 5143 * without taking a lock. 5144 */ 5145 private volatile boolean resourcesLow = false; 5146 /** Should safemode adjust its block totals as blocks come in */ 5147 private boolean shouldIncrementallyTrackBlocks = false; 5148 /** counter for tracking startup progress of reported blocks */ 5149 private Counter awaitingReportedBlocksCounter; 5150 5151 /** 5152 * Creates SafeModeInfo when the name node enters 5153 * automatic safe mode at startup. 5154 * 5155 * @param conf configuration 5156 */ 5157 private SafeModeInfo(Configuration conf) { 5158 this.threshold = conf.getFloat(DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 5159 DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_DEFAULT); 5160 if(threshold > 1.0) { 5161 LOG.warn("The threshold value should't be greater than 1, threshold: " + threshold); 5162 } 5163 this.datanodeThreshold = conf.getInt( 5164 DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY, 5165 DFS_NAMENODE_SAFEMODE_MIN_DATANODES_DEFAULT); 5166 this.extension = conf.getInt(DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0); 5167 this.safeReplication = conf.getInt(DFS_NAMENODE_REPLICATION_MIN_KEY, 5168 DFS_NAMENODE_REPLICATION_MIN_DEFAULT); 5169 5170 LOG.info(DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY + " = " + threshold); 5171 LOG.info(DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY + " = " + datanodeThreshold); 5172 LOG.info(DFS_NAMENODE_SAFEMODE_EXTENSION_KEY + " = " + extension); 5173 5174 // default to safe mode threshold (i.e., don't populate queues before leaving safe mode) 5175 this.replQueueThreshold = 5176 conf.getFloat(DFS_NAMENODE_REPL_QUEUE_THRESHOLD_PCT_KEY, 5177 (float) threshold); 5178 this.blockTotal = 0; 5179 this.blockSafe = 0; 5180 } 5181 5182 /** 5183 * In the HA case, the StandbyNode can be in safemode while the namespace 5184 * is modified by the edit log tailer. In this case, the number of total 5185 * blocks changes as edits are processed (eg blocks are added and deleted). 5186 * However, we don't want to do the incremental tracking during the 5187 * startup-time loading process -- only once the initial total has been 5188 * set after the image has been loaded. 5189 */ 5190 private boolean shouldIncrementallyTrackBlocks() { 5191 return shouldIncrementallyTrackBlocks; 5192 } 5193 5194 /** 5195 * Creates SafeModeInfo when safe mode is entered manually, or because 5196 * available resources are low. 5197 * 5198 * The {@link #threshold} is set to 1.5 so that it could never be reached. 5199 * {@link #blockTotal} is set to -1 to indicate that safe mode is manual. 5200 * 5201 * @see SafeModeInfo 5202 */ 5203 private SafeModeInfo(boolean resourcesLow) { 5204 this.threshold = 1.5f; // this threshold can never be reached 5205 this.datanodeThreshold = Integer.MAX_VALUE; 5206 this.extension = Integer.MAX_VALUE; 5207 this.safeReplication = Short.MAX_VALUE + 1; // more than maxReplication 5208 this.replQueueThreshold = 1.5f; // can never be reached 5209 this.blockTotal = -1; 5210 this.blockSafe = -1; 5211 this.resourcesLow = resourcesLow; 5212 enter(); 5213 reportStatus("STATE* Safe mode is ON.", true); 5214 } 5215 5216 /** 5217 * Check if safe mode is on. 5218 * @return true if in safe mode 5219 */ 5220 private synchronized boolean isOn() { 5221 doConsistencyCheck(); 5222 return this.reached >= 0; 5223 } 5224 5225 /** 5226 * Enter safe mode. 5227 */ 5228 private void enter() { 5229 this.reached = 0; 5230 this.reachedTimestamp = 0; 5231 } 5232 5233 /** 5234 * Leave safe mode. 5235 * <p> 5236 * Check for invalid, under- & over-replicated blocks in the end of startup. 5237 */ 5238 private synchronized void leave() { 5239 // if not done yet, initialize replication queues. 5240 // In the standby, do not populate repl queues 5241 if (!isPopulatingReplQueues() && shouldPopulateReplQueues()) { 5242 initializeReplQueues(); 5243 } 5244 long timeInSafemode = now() - startTime; 5245 NameNode.stateChangeLog.info("STATE* Leaving safe mode after " 5246 + timeInSafemode/1000 + " secs"); 5247 NameNode.getNameNodeMetrics().setSafeModeTime((int) timeInSafemode); 5248 5249 //Log the following only once (when transitioning from ON -> OFF) 5250 if (reached >= 0) { 5251 NameNode.stateChangeLog.info("STATE* Safe mode is OFF"); 5252 } 5253 reached = -1; 5254 reachedTimestamp = -1; 5255 safeMode = null; 5256 final NetworkTopology nt = blockManager.getDatanodeManager().getNetworkTopology(); 5257 NameNode.stateChangeLog.info("STATE* Network topology has " 5258 + nt.getNumOfRacks() + " racks and " 5259 + nt.getNumOfLeaves() + " datanodes"); 5260 NameNode.stateChangeLog.info("STATE* UnderReplicatedBlocks has " 5261 + blockManager.numOfUnderReplicatedBlocks() + " blocks"); 5262 5263 startSecretManagerIfNecessary(); 5264 5265 // If startup has not yet completed, end safemode phase. 5266 StartupProgress prog = NameNode.getStartupProgress(); 5267 if (prog.getStatus(Phase.SAFEMODE) != Status.COMPLETE) { 5268 prog.endStep(Phase.SAFEMODE, STEP_AWAITING_REPORTED_BLOCKS); 5269 prog.endPhase(Phase.SAFEMODE); 5270 } 5271 } 5272 5273 /** 5274 * Check whether we have reached the threshold for 5275 * initializing replication queues. 5276 */ 5277 private synchronized boolean canInitializeReplQueues() { 5278 return shouldPopulateReplQueues() 5279 && blockSafe >= blockReplQueueThreshold; 5280 } 5281 5282 /** 5283 * Safe mode can be turned off iff 5284 * the threshold is reached and 5285 * the extension time have passed. 5286 * @return true if can leave or false otherwise. 5287 */ 5288 private synchronized boolean canLeave() { 5289 if (reached == 0) { 5290 return false; 5291 } 5292 5293 if (monotonicNow() - reached < extension) { 5294 reportStatus("STATE* Safe mode ON, in safe mode extension.", false); 5295 return false; 5296 } 5297 5298 if (needEnter()) { 5299 reportStatus("STATE* Safe mode ON, thresholds not met.", false); 5300 return false; 5301 } 5302 5303 return true; 5304 } 5305 5306 /** 5307 * There is no need to enter safe mode 5308 * if DFS is empty or {@link #threshold} == 0 5309 */ 5310 private boolean needEnter() { 5311 return (threshold != 0 && blockSafe < blockThreshold) || 5312 (datanodeThreshold != 0 && getNumLiveDataNodes() < datanodeThreshold) || 5313 (!nameNodeHasResourcesAvailable()); 5314 } 5315 5316 /** 5317 * Check and trigger safe mode if needed. 5318 */ 5319 private void checkMode() { 5320 // Have to have write-lock since leaving safemode initializes 5321 // repl queues, which requires write lock 5322 assert hasWriteLock(); 5323 if (inTransitionToActive()) { 5324 return; 5325 } 5326 // if smmthread is already running, the block threshold must have been 5327 // reached before, there is no need to enter the safe mode again 5328 if (smmthread == null && needEnter()) { 5329 enter(); 5330 // check if we are ready to initialize replication queues 5331 if (canInitializeReplQueues() && !isPopulatingReplQueues() 5332 && !haEnabled) { 5333 initializeReplQueues(); 5334 } 5335 reportStatus("STATE* Safe mode ON.", false); 5336 return; 5337 } 5338 // the threshold is reached or was reached before 5339 if (!isOn() || // safe mode is off 5340 extension <= 0 || threshold <= 0) { // don't need to wait 5341 this.leave(); // leave safe mode 5342 return; 5343 } 5344 if (reached > 0) { // threshold has already been reached before 5345 reportStatus("STATE* Safe mode ON.", false); 5346 return; 5347 } 5348 // start monitor 5349 reached = monotonicNow(); 5350 reachedTimestamp = now(); 5351 if (smmthread == null) { 5352 smmthread = new Daemon(new SafeModeMonitor()); 5353 smmthread.start(); 5354 reportStatus("STATE* Safe mode extension entered.", true); 5355 } 5356 5357 // check if we are ready to initialize replication queues 5358 if (canInitializeReplQueues() && !isPopulatingReplQueues() && !haEnabled) { 5359 initializeReplQueues(); 5360 } 5361 } 5362 5363 /** 5364 * Set total number of blocks. 5365 */ 5366 private synchronized void setBlockTotal(int total) { 5367 this.blockTotal = total; 5368 this.blockThreshold = (int) (blockTotal * threshold); 5369 this.blockReplQueueThreshold = 5370 (int) (blockTotal * replQueueThreshold); 5371 if (haEnabled) { 5372 // After we initialize the block count, any further namespace 5373 // modifications done while in safe mode need to keep track 5374 // of the number of total blocks in the system. 5375 this.shouldIncrementallyTrackBlocks = true; 5376 } 5377 if(blockSafe < 0) 5378 this.blockSafe = 0; 5379 checkMode(); 5380 } 5381 5382 /** 5383 * Increment number of safe blocks if current block has 5384 * reached minimal replication. 5385 * @param replication current replication 5386 */ 5387 private synchronized void incrementSafeBlockCount(short replication) { 5388 if (replication == safeReplication) { 5389 this.blockSafe++; 5390 5391 // Report startup progress only if we haven't completed startup yet. 5392 StartupProgress prog = NameNode.getStartupProgress(); 5393 if (prog.getStatus(Phase.SAFEMODE) != Status.COMPLETE) { 5394 if (this.awaitingReportedBlocksCounter == null) { 5395 this.awaitingReportedBlocksCounter = prog.getCounter(Phase.SAFEMODE, 5396 STEP_AWAITING_REPORTED_BLOCKS); 5397 } 5398 this.awaitingReportedBlocksCounter.increment(); 5399 } 5400 5401 checkMode(); 5402 } 5403 } 5404 5405 /** 5406 * Decrement number of safe blocks if current block has 5407 * fallen below minimal replication. 5408 * @param replication current replication 5409 */ 5410 private synchronized void decrementSafeBlockCount(short replication) { 5411 if (replication == safeReplication-1) { 5412 this.blockSafe--; 5413 //blockSafe is set to -1 in manual / low resources safemode 5414 assert blockSafe >= 0 || isManual() || areResourcesLow(); 5415 checkMode(); 5416 } 5417 } 5418 5419 /** 5420 * Check if safe mode was entered manually 5421 */ 5422 private boolean isManual() { 5423 return extension == Integer.MAX_VALUE; 5424 } 5425 5426 /** 5427 * Set manual safe mode. 5428 */ 5429 private synchronized void setManual() { 5430 extension = Integer.MAX_VALUE; 5431 } 5432 5433 /** 5434 * Check if safe mode was entered due to resources being low. 5435 */ 5436 private boolean areResourcesLow() { 5437 return resourcesLow; 5438 } 5439 5440 /** 5441 * Set that resources are low for this instance of safe mode. 5442 */ 5443 private void setResourcesLow() { 5444 resourcesLow = true; 5445 } 5446 5447 /** 5448 * A tip on how safe mode is to be turned off: manually or automatically. 5449 */ 5450 String getTurnOffTip() { 5451 if(!isOn()) { 5452 return "Safe mode is OFF."; 5453 } 5454 5455 //Manual OR low-resource safemode. (Admin intervention required) 5456 String adminMsg = "It was turned on manually. "; 5457 if (areResourcesLow()) { 5458 adminMsg = "Resources are low on NN. Please add or free up more " 5459 + "resources then turn off safe mode manually. NOTE: If you turn off" 5460 + " safe mode before adding resources, " 5461 + "the NN will immediately return to safe mode. "; 5462 } 5463 if (isManual() || areResourcesLow()) { 5464 return adminMsg 5465 + "Use \"hdfs dfsadmin -safemode leave\" to turn safe mode off."; 5466 } 5467 5468 boolean thresholdsMet = true; 5469 int numLive = getNumLiveDataNodes(); 5470 String msg = ""; 5471 if (blockSafe < blockThreshold) { 5472 msg += String.format( 5473 "The reported blocks %d needs additional %d" 5474 + " blocks to reach the threshold %.4f of total blocks %d.%n", 5475 blockSafe, (blockThreshold - blockSafe) + 1, threshold, blockTotal); 5476 thresholdsMet = false; 5477 } else { 5478 msg += String.format("The reported blocks %d has reached the threshold" 5479 + " %.4f of total blocks %d. ", blockSafe, threshold, blockTotal); 5480 } 5481 if (numLive < datanodeThreshold) { 5482 msg += String.format( 5483 "The number of live datanodes %d needs an additional %d live " 5484 + "datanodes to reach the minimum number %d.%n", 5485 numLive, (datanodeThreshold - numLive), datanodeThreshold); 5486 thresholdsMet = false; 5487 } else { 5488 msg += String.format("The number of live datanodes %d has reached " 5489 + "the minimum number %d. ", 5490 numLive, datanodeThreshold); 5491 } 5492 msg += (reached > 0) ? "In safe mode extension. " : ""; 5493 msg += "Safe mode will be turned off automatically "; 5494 5495 if (!thresholdsMet) { 5496 msg += "once the thresholds have been reached."; 5497 } else if (reached + extension - monotonicNow() > 0) { 5498 msg += ("in " + (reached + extension - monotonicNow()) / 1000 + " seconds."); 5499 } else { 5500 msg += "soon."; 5501 } 5502 5503 return msg; 5504 } 5505 5506 /** 5507 * Print status every 20 seconds. 5508 */ 5509 private void reportStatus(String msg, boolean rightNow) { 5510 long curTime = now(); 5511 if(!rightNow && (curTime - lastStatusReport < 20 * 1000)) 5512 return; 5513 NameNode.stateChangeLog.info(msg + " \n" + getTurnOffTip()); 5514 lastStatusReport = curTime; 5515 } 5516 5517 @Override 5518 public String toString() { 5519 String resText = "Current safe blocks = " 5520 + blockSafe 5521 + ". Target blocks = " + blockThreshold + " for threshold = %" + threshold 5522 + ". Minimal replication = " + safeReplication + "."; 5523 if (reached > 0) 5524 resText += " Threshold was reached " + new Date(reachedTimestamp) + "."; 5525 return resText; 5526 } 5527 5528 /** 5529 * Checks consistency of the class state. 5530 * This is costly so only runs if asserts are enabled. 5531 */ 5532 private void doConsistencyCheck() { 5533 boolean assertsOn = false; 5534 assert assertsOn = true; // set to true if asserts are on 5535 if (!assertsOn) return; 5536 5537 if (blockTotal == -1 && blockSafe == -1) { 5538 return; // manual safe mode 5539 } 5540 int activeBlocks = blockManager.getActiveBlockCount(); 5541 if ((blockTotal != activeBlocks) && 5542 !(blockSafe >= 0 && blockSafe <= blockTotal)) { 5543 throw new AssertionError( 5544 " SafeMode: Inconsistent filesystem state: " 5545 + "SafeMode data: blockTotal=" + blockTotal 5546 + " blockSafe=" + blockSafe + "; " 5547 + "BlockManager data: active=" + activeBlocks); 5548 } 5549 } 5550 5551 private synchronized void adjustBlockTotals(int deltaSafe, int deltaTotal) { 5552 if (!shouldIncrementallyTrackBlocks) { 5553 return; 5554 } 5555 assert haEnabled; 5556 5557 if (LOG.isDebugEnabled()) { 5558 LOG.debug("Adjusting block totals from " + 5559 blockSafe + "/" + blockTotal + " to " + 5560 (blockSafe + deltaSafe) + "/" + (blockTotal + deltaTotal)); 5561 } 5562 assert blockSafe + deltaSafe >= 0 : "Can't reduce blockSafe " + 5563 blockSafe + " by " + deltaSafe + ": would be negative"; 5564 assert blockTotal + deltaTotal >= 0 : "Can't reduce blockTotal " + 5565 blockTotal + " by " + deltaTotal + ": would be negative"; 5566 5567 blockSafe += deltaSafe; 5568 setBlockTotal(blockTotal + deltaTotal); 5569 } 5570 } 5571 5572 /** 5573 * Periodically check whether it is time to leave safe mode. 5574 * This thread starts when the threshold level is reached. 5575 * 5576 */ 5577 class SafeModeMonitor implements Runnable { 5578 /** interval in msec for checking safe mode: {@value} */ 5579 private static final long recheckInterval = 1000; 5580 5581 /** 5582 */ 5583 @Override 5584 public void run() { 5585 while (fsRunning) { 5586 writeLock(); 5587 try { 5588 if (safeMode == null) { // Not in safe mode. 5589 break; 5590 } 5591 if (safeMode.canLeave()) { 5592 // Leave safe mode. 5593 safeMode.leave(); 5594 smmthread = null; 5595 break; 5596 } 5597 } finally { 5598 writeUnlock(); 5599 } 5600 5601 try { 5602 Thread.sleep(recheckInterval); 5603 } catch (InterruptedException ie) { 5604 // Ignored 5605 } 5606 } 5607 if (!fsRunning) { 5608 LOG.info("NameNode is being shutdown, exit SafeModeMonitor thread"); 5609 } 5610 } 5611 } 5612 5613 boolean setSafeMode(SafeModeAction action) throws IOException { 5614 if (action != SafeModeAction.SAFEMODE_GET) { 5615 checkSuperuserPrivilege(); 5616 switch(action) { 5617 case SAFEMODE_LEAVE: // leave safe mode 5618 leaveSafeMode(); 5619 break; 5620 case SAFEMODE_ENTER: // enter safe mode 5621 enterSafeMode(false); 5622 break; 5623 default: 5624 LOG.error("Unexpected safe mode action"); 5625 } 5626 } 5627 return isInSafeMode(); 5628 } 5629 5630 @Override 5631 public void checkSafeMode() { 5632 // safeMode is volatile, and may be set to null at any time 5633 SafeModeInfo safeMode = this.safeMode; 5634 if (safeMode != null) { 5635 safeMode.checkMode(); 5636 } 5637 } 5638 5639 @Override 5640 public boolean isInSafeMode() { 5641 // safeMode is volatile, and may be set to null at any time 5642 SafeModeInfo safeMode = this.safeMode; 5643 if (safeMode == null) 5644 return false; 5645 return safeMode.isOn(); 5646 } 5647 5648 @Override 5649 public boolean isInStartupSafeMode() { 5650 // safeMode is volatile, and may be set to null at any time 5651 SafeModeInfo safeMode = this.safeMode; 5652 if (safeMode == null) 5653 return false; 5654 // If the NN is in safemode, and not due to manual / low resources, we 5655 // assume it must be because of startup. If the NN had low resources during 5656 // startup, we assume it came out of startup safemode and it is now in low 5657 // resources safemode 5658 return !safeMode.isManual() && !safeMode.areResourcesLow() 5659 && safeMode.isOn(); 5660 } 5661 5662 /** 5663 * Check if replication queues are to be populated 5664 * @return true when node is HAState.Active and not in the very first safemode 5665 */ 5666 @Override 5667 public boolean isPopulatingReplQueues() { 5668 if (!shouldPopulateReplQueues()) { 5669 return false; 5670 } 5671 return initializedReplQueues; 5672 } 5673 5674 private boolean shouldPopulateReplQueues() { 5675 if(haContext == null || haContext.getState() == null) 5676 return false; 5677 return haContext.getState().shouldPopulateReplQueues(); 5678 } 5679 5680 @Override 5681 public void incrementSafeBlockCount(int replication) { 5682 // safeMode is volatile, and may be set to null at any time 5683 SafeModeInfo safeMode = this.safeMode; 5684 if (safeMode == null) 5685 return; 5686 safeMode.incrementSafeBlockCount((short)replication); 5687 } 5688 5689 @Override 5690 public void decrementSafeBlockCount(Block b) { 5691 // safeMode is volatile, and may be set to null at any time 5692 SafeModeInfo safeMode = this.safeMode; 5693 if (safeMode == null) // mostly true 5694 return; 5695 BlockInfoContiguous storedBlock = getStoredBlock(b); 5696 if (storedBlock.isComplete()) { 5697 safeMode.decrementSafeBlockCount((short)blockManager.countNodes(b).liveReplicas()); 5698 } 5699 } 5700 5701 /** 5702 * Adjust the total number of blocks safe and expected during safe mode. 5703 * If safe mode is not currently on, this is a no-op. 5704 * @param deltaSafe the change in number of safe blocks 5705 * @param deltaTotal the change i nnumber of total blocks expected 5706 */ 5707 @Override 5708 public void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal) { 5709 // safeMode is volatile, and may be set to null at any time 5710 SafeModeInfo safeMode = this.safeMode; 5711 if (safeMode == null) 5712 return; 5713 safeMode.adjustBlockTotals(deltaSafe, deltaTotal); 5714 } 5715 5716 /** 5717 * Set the total number of blocks in the system. 5718 */ 5719 public void setBlockTotal() { 5720 // safeMode is volatile, and may be set to null at any time 5721 SafeModeInfo safeMode = this.safeMode; 5722 if (safeMode == null) 5723 return; 5724 safeMode.setBlockTotal((int)getCompleteBlocksTotal()); 5725 } 5726 5727 /** 5728 * Get the total number of blocks in the system. 5729 */ 5730 @Override // FSNamesystemMBean 5731 @Metric 5732 public long getBlocksTotal() { 5733 return blockManager.getTotalBlocks(); 5734 } 5735 5736 /** 5737 * Get the total number of COMPLETE blocks in the system. 5738 * For safe mode only complete blocks are counted. 5739 */ 5740 private long getCompleteBlocksTotal() { 5741 // Calculate number of blocks under construction 5742 long numUCBlocks = 0; 5743 readLock(); 5744 numUCBlocks = leaseManager.getNumUnderConstructionBlocks(); 5745 try { 5746 return getBlocksTotal() - numUCBlocks; 5747 } finally { 5748 readUnlock(); 5749 } 5750 } 5751 5752 /** 5753 * Enter safe mode. If resourcesLow is false, then we assume it is manual 5754 * @throws IOException 5755 */ 5756 void enterSafeMode(boolean resourcesLow) throws IOException { 5757 writeLock(); 5758 try { 5759 // Stop the secret manager, since rolling the master key would 5760 // try to write to the edit log 5761 stopSecretManager(); 5762 5763 // Ensure that any concurrent operations have been fully synced 5764 // before entering safe mode. This ensures that the FSImage 5765 // is entirely stable on disk as soon as we're in safe mode. 5766 boolean isEditlogOpenForWrite = getEditLog().isOpenForWrite(); 5767 // Before Editlog is in OpenForWrite mode, editLogStream will be null. So, 5768 // logSyncAll call can be called only when Edlitlog is in OpenForWrite mode 5769 if (isEditlogOpenForWrite) { 5770 getEditLog().logSyncAll(); 5771 } 5772 if (!isInSafeMode()) { 5773 safeMode = new SafeModeInfo(resourcesLow); 5774 return; 5775 } 5776 if (resourcesLow) { 5777 safeMode.setResourcesLow(); 5778 } else { 5779 safeMode.setManual(); 5780 } 5781 if (isEditlogOpenForWrite) { 5782 getEditLog().logSyncAll(); 5783 } 5784 NameNode.stateChangeLog.info("STATE* Safe mode is ON" 5785 + safeMode.getTurnOffTip()); 5786 } finally { 5787 writeUnlock(); 5788 } 5789 } 5790 5791 /** 5792 * Leave safe mode. 5793 */ 5794 void leaveSafeMode() { 5795 writeLock(); 5796 try { 5797 if (!isInSafeMode()) { 5798 NameNode.stateChangeLog.info("STATE* Safe mode is already OFF"); 5799 return; 5800 } 5801 safeMode.leave(); 5802 } finally { 5803 writeUnlock(); 5804 } 5805 } 5806 5807 String getSafeModeTip() { 5808 // There is no need to take readLock. 5809 // Don't use isInSafeMode as this.safeMode might be set to null. 5810 // after isInSafeMode returns. 5811 boolean inSafeMode; 5812 SafeModeInfo safeMode = this.safeMode; 5813 if (safeMode == null) { 5814 inSafeMode = false; 5815 } else { 5816 inSafeMode = safeMode.isOn(); 5817 } 5818 5819 if (!inSafeMode) { 5820 return ""; 5821 } else { 5822 return safeMode.getTurnOffTip(); 5823 } 5824 } 5825 5826 CheckpointSignature rollEditLog() throws IOException { 5827 checkSuperuserPrivilege(); 5828 checkOperation(OperationCategory.JOURNAL); 5829 writeLock(); 5830 try { 5831 checkOperation(OperationCategory.JOURNAL); 5832 checkNameNodeSafeMode("Log not rolled"); 5833 if (Server.isRpcInvocation()) { 5834 LOG.info("Roll Edit Log from " + Server.getRemoteAddress()); 5835 } 5836 return getFSImage().rollEditLog(); 5837 } finally { 5838 writeUnlock(); 5839 } 5840 } 5841 5842 NamenodeCommand startCheckpoint(NamenodeRegistration backupNode, 5843 NamenodeRegistration activeNamenode) throws IOException { 5844 checkOperation(OperationCategory.CHECKPOINT); 5845 writeLock(); 5846 try { 5847 checkOperation(OperationCategory.CHECKPOINT); 5848 checkNameNodeSafeMode("Checkpoint not started"); 5849 5850 LOG.info("Start checkpoint for " + backupNode.getAddress()); 5851 NamenodeCommand cmd = getFSImage().startCheckpoint(backupNode, 5852 activeNamenode); 5853 getEditLog().logSync(); 5854 return cmd; 5855 } finally { 5856 writeUnlock(); 5857 } 5858 } 5859 5860 public void processIncrementalBlockReport(final DatanodeID nodeID, 5861 final StorageReceivedDeletedBlocks srdb) 5862 throws IOException { 5863 writeLock(); 5864 try { 5865 blockManager.processIncrementalBlockReport(nodeID, srdb); 5866 } finally { 5867 writeUnlock(); 5868 } 5869 } 5870 5871 void endCheckpoint(NamenodeRegistration registration, 5872 CheckpointSignature sig) throws IOException { 5873 checkOperation(OperationCategory.CHECKPOINT); 5874 readLock(); 5875 try { 5876 checkOperation(OperationCategory.CHECKPOINT); 5877 checkNameNodeSafeMode("Checkpoint not ended"); 5878 LOG.info("End checkpoint for " + registration.getAddress()); 5879 getFSImage().endCheckpoint(sig); 5880 } finally { 5881 readUnlock(); 5882 } 5883 } 5884 5885 PermissionStatus createFsOwnerPermissions(FsPermission permission) { 5886 return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission); 5887 } 5888 5889 private void checkUnreadableBySuperuser(FSPermissionChecker pc, 5890 INode inode, int snapshotId) 5891 throws IOException { 5892 if (pc.isSuperUser()) { 5893 for (XAttr xattr : FSDirXAttrOp.getXAttrs(dir, inode, snapshotId)) { 5894 if (XAttrHelper.getPrefixName(xattr). 5895 equals(SECURITY_XATTR_UNREADABLE_BY_SUPERUSER)) { 5896 throw new AccessControlException("Access is denied for " + 5897 pc.getUser() + " since the superuser is not allowed to " + 5898 "perform this operation."); 5899 } 5900 } 5901 } 5902 } 5903 5904 @Override 5905 public void checkSuperuserPrivilege() 5906 throws AccessControlException { 5907 if (isPermissionEnabled) { 5908 FSPermissionChecker pc = getPermissionChecker(); 5909 pc.checkSuperuserPrivilege(); 5910 } 5911 } 5912 5913 /** 5914 * Check to see if we have exceeded the limit on the number 5915 * of inodes. 5916 */ 5917 void checkFsObjectLimit() throws IOException { 5918 if (maxFsObjects != 0 && 5919 maxFsObjects <= dir.totalInodes() + getBlocksTotal()) { 5920 throw new IOException("Exceeded the configured number of objects " + 5921 maxFsObjects + " in the filesystem."); 5922 } 5923 } 5924 5925 /** 5926 * Get the total number of objects in the system. 5927 */ 5928 @Override // FSNamesystemMBean 5929 public long getMaxObjects() { 5930 return maxFsObjects; 5931 } 5932 5933 @Override // FSNamesystemMBean 5934 @Metric 5935 public long getFilesTotal() { 5936 // There is no need to take fSNamesystem's lock as 5937 // FSDirectory has its own lock. 5938 return this.dir.totalInodes(); 5939 } 5940 5941 @Override // FSNamesystemMBean 5942 @Metric 5943 public long getPendingReplicationBlocks() { 5944 return blockManager.getPendingReplicationBlocksCount(); 5945 } 5946 5947 @Override // FSNamesystemMBean 5948 @Metric 5949 public long getUnderReplicatedBlocks() { 5950 return blockManager.getUnderReplicatedBlocksCount(); 5951 } 5952 5953 /** Returns number of blocks with corrupt replicas */ 5954 @Metric({"CorruptBlocks", "Number of blocks with corrupt replicas"}) 5955 public long getCorruptReplicaBlocks() { 5956 return blockManager.getCorruptReplicaBlocksCount(); 5957 } 5958 5959 @Override // FSNamesystemMBean 5960 @Metric 5961 public long getScheduledReplicationBlocks() { 5962 return blockManager.getScheduledReplicationBlocksCount(); 5963 } 5964 5965 @Override 5966 @Metric 5967 public long getPendingDeletionBlocks() { 5968 return blockManager.getPendingDeletionBlocksCount(); 5969 } 5970 5971 @Override 5972 public long getBlockDeletionStartTime() { 5973 return startTime + blockManager.getStartupDelayBlockDeletionInMs(); 5974 } 5975 5976 @Metric 5977 public long getExcessBlocks() { 5978 return blockManager.getExcessBlocksCount(); 5979 } 5980 5981 // HA-only metric 5982 @Metric 5983 public long getPostponedMisreplicatedBlocks() { 5984 return blockManager.getPostponedMisreplicatedBlocksCount(); 5985 } 5986 5987 // HA-only metric 5988 @Metric 5989 public int getPendingDataNodeMessageCount() { 5990 return blockManager.getPendingDataNodeMessageCount(); 5991 } 5992 5993 // HA-only metric 5994 @Metric 5995 public String getHAState() { 5996 return haContext.getState().toString(); 5997 } 5998 5999 // HA-only metric 6000 @Metric 6001 public long getMillisSinceLastLoadedEdits() { 6002 if (isInStandbyState() && editLogTailer != null) { 6003 return monotonicNow() - editLogTailer.getLastLoadTimeMs(); 6004 } else { 6005 return 0; 6006 } 6007 } 6008 6009 @Metric 6010 public int getBlockCapacity() { 6011 return blockManager.getCapacity(); 6012 } 6013 6014 @Override // FSNamesystemMBean 6015 public String getFSState() { 6016 return isInSafeMode() ? "safeMode" : "Operational"; 6017 } 6018 6019 private ObjectName mbeanName; 6020 private ObjectName mxbeanName; 6021 6022 /** 6023 * Register the FSNamesystem MBean using the name 6024 * "hadoop:service=NameNode,name=FSNamesystemState" 6025 */ 6026 private void registerMBean() { 6027 // We can only implement one MXBean interface, so we keep the old one. 6028 try { 6029 StandardMBean bean = new StandardMBean(this, FSNamesystemMBean.class); 6030 mbeanName = MBeans.register("NameNode", "FSNamesystemState", bean); 6031 } catch (NotCompliantMBeanException e) { 6032 throw new RuntimeException("Bad MBean setup", e); 6033 } 6034 6035 LOG.info("Registered FSNamesystemState MBean"); 6036 } 6037 6038 /** 6039 * shutdown FSNamesystem 6040 */ 6041 void shutdown() { 6042 if (snapshotManager != null) { 6043 snapshotManager.shutdown(); 6044 } 6045 if (mbeanName != null) { 6046 MBeans.unregister(mbeanName); 6047 mbeanName = null; 6048 } 6049 if (mxbeanName != null) { 6050 MBeans.unregister(mxbeanName); 6051 mxbeanName = null; 6052 } 6053 if (dir != null) { 6054 dir.shutdown(); 6055 } 6056 if (blockManager != null) { 6057 blockManager.shutdown(); 6058 } 6059 } 6060 6061 @Override // FSNamesystemMBean 6062 public int getNumLiveDataNodes() { 6063 return getBlockManager().getDatanodeManager().getNumLiveDataNodes(); 6064 } 6065 6066 @Override // FSNamesystemMBean 6067 public int getNumDeadDataNodes() { 6068 return getBlockManager().getDatanodeManager().getNumDeadDataNodes(); 6069 } 6070 6071 @Override // FSNamesystemMBean 6072 public int getNumDecomLiveDataNodes() { 6073 final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); 6074 getBlockManager().getDatanodeManager().fetchDatanodes(live, null, true); 6075 int liveDecommissioned = 0; 6076 for (DatanodeDescriptor node : live) { 6077 liveDecommissioned += node.isDecommissioned() ? 1 : 0; 6078 } 6079 return liveDecommissioned; 6080 } 6081 6082 @Override // FSNamesystemMBean 6083 public int getNumDecomDeadDataNodes() { 6084 final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>(); 6085 getBlockManager().getDatanodeManager().fetchDatanodes(null, dead, true); 6086 int deadDecommissioned = 0; 6087 for (DatanodeDescriptor node : dead) { 6088 deadDecommissioned += node.isDecommissioned() ? 1 : 0; 6089 } 6090 return deadDecommissioned; 6091 } 6092 6093 @Override // FSNamesystemMBean 6094 public int getVolumeFailuresTotal() { 6095 List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); 6096 getBlockManager().getDatanodeManager().fetchDatanodes(live, null, true); 6097 int volumeFailuresTotal = 0; 6098 for (DatanodeDescriptor node: live) { 6099 volumeFailuresTotal += node.getVolumeFailures(); 6100 } 6101 return volumeFailuresTotal; 6102 } 6103 6104 @Override // FSNamesystemMBean 6105 public long getEstimatedCapacityLostTotal() { 6106 List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); 6107 getBlockManager().getDatanodeManager().fetchDatanodes(live, null, true); 6108 long estimatedCapacityLostTotal = 0; 6109 for (DatanodeDescriptor node: live) { 6110 VolumeFailureSummary volumeFailureSummary = node.getVolumeFailureSummary(); 6111 if (volumeFailureSummary != null) { 6112 estimatedCapacityLostTotal += 6113 volumeFailureSummary.getEstimatedCapacityLostTotal(); 6114 } 6115 } 6116 return estimatedCapacityLostTotal; 6117 } 6118 6119 @Override // FSNamesystemMBean 6120 public int getNumDecommissioningDataNodes() { 6121 return getBlockManager().getDatanodeManager().getDecommissioningNodes() 6122 .size(); 6123 } 6124 6125 @Override // FSNamesystemMBean 6126 @Metric({"StaleDataNodes", 6127 "Number of datanodes marked stale due to delayed heartbeat"}) 6128 public int getNumStaleDataNodes() { 6129 return getBlockManager().getDatanodeManager().getNumStaleNodes(); 6130 } 6131 6132 /** 6133 * Storages are marked as "content stale" after NN restart or fails over and 6134 * before NN receives the first Heartbeat followed by the first Blockreport. 6135 */ 6136 @Override // FSNamesystemMBean 6137 public int getNumStaleStorages() { 6138 return getBlockManager().getDatanodeManager().getNumStaleStorages(); 6139 } 6140 6141 @Override // FSNamesystemMBean 6142 public String getTopUserOpCounts() { 6143 if (!topConf.isEnabled) { 6144 return null; 6145 } 6146 6147 Date now = new Date(); 6148 final List<RollingWindowManager.TopWindow> topWindows = 6149 topMetrics.getTopWindows(); 6150 Map<String, Object> topMap = new TreeMap<String, Object>(); 6151 topMap.put("windows", topWindows); 6152 topMap.put("timestamp", DFSUtil.dateToIso8601String(now)); 6153 ObjectMapper mapper = new ObjectMapper(); 6154 try { 6155 return mapper.writeValueAsString(topMap); 6156 } catch (IOException e) { 6157 LOG.warn("Failed to fetch TopUser metrics", e); 6158 } 6159 return null; 6160 } 6161 6162 /** 6163 * Increments, logs and then returns the stamp 6164 */ 6165 long nextGenerationStamp(boolean legacyBlock) 6166 throws IOException, SafeModeException { 6167 assert hasWriteLock(); 6168 checkNameNodeSafeMode("Cannot get next generation stamp"); 6169 6170 long gs = blockIdManager.nextGenerationStamp(legacyBlock); 6171 if (legacyBlock) { 6172 getEditLog().logGenerationStampV1(gs); 6173 } else { 6174 getEditLog().logGenerationStampV2(gs); 6175 } 6176 6177 // NB: callers sync the log 6178 return gs; 6179 } 6180 6181 /** 6182 * Increments, logs and then returns the block ID 6183 */ 6184 private long nextBlockId() throws IOException { 6185 assert hasWriteLock(); 6186 checkNameNodeSafeMode("Cannot get next block ID"); 6187 final long blockId = blockIdManager.nextBlockId(); 6188 getEditLog().logAllocateBlockId(blockId); 6189 // NB: callers sync the log 6190 return blockId; 6191 } 6192 6193 private boolean isFileDeleted(INodeFile file) { 6194 // Not in the inodeMap or in the snapshot but marked deleted. 6195 if (dir.getInode(file.getId()) == null) { 6196 return true; 6197 } 6198 6199 // look at the path hierarchy to see if one parent is deleted by recursive 6200 // deletion 6201 INode tmpChild = file; 6202 INodeDirectory tmpParent = file.getParent(); 6203 while (true) { 6204 if (tmpParent == null) { 6205 return true; 6206 } 6207 6208 INode childINode = tmpParent.getChild(tmpChild.getLocalNameBytes(), 6209 Snapshot.CURRENT_STATE_ID); 6210 if (childINode == null || !childINode.equals(tmpChild)) { 6211 // a newly created INode with the same name as an already deleted one 6212 // would be a different INode than the deleted one 6213 return true; 6214 } 6215 6216 if (tmpParent.isRoot()) { 6217 break; 6218 } 6219 6220 tmpChild = tmpParent; 6221 tmpParent = tmpParent.getParent(); 6222 } 6223 6224 if (file.isWithSnapshot() && 6225 file.getFileWithSnapshotFeature().isCurrentFileDeleted()) { 6226 return true; 6227 } 6228 return false; 6229 } 6230 6231 private INodeFile checkUCBlock(ExtendedBlock block, 6232 String clientName) throws IOException { 6233 assert hasWriteLock(); 6234 checkNameNodeSafeMode("Cannot get a new generation stamp and an " 6235 + "access token for block " + block); 6236 6237 // check stored block state 6238 BlockInfoContiguous storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block)); 6239 if (storedBlock == null || 6240 storedBlock.getBlockUCState() != BlockUCState.UNDER_CONSTRUCTION) { 6241 throw new IOException(block + 6242 " does not exist or is not under Construction" + storedBlock); 6243 } 6244 6245 // check file inode 6246 final INodeFile file = ((INode)storedBlock.getBlockCollection()).asFile(); 6247 if (file == null || !file.isUnderConstruction() || isFileDeleted(file)) { 6248 throw new IOException("The file " + storedBlock + 6249 " belonged to does not exist or it is not under construction."); 6250 } 6251 6252 // check lease 6253 if (clientName == null 6254 || !clientName.equals(file.getFileUnderConstructionFeature() 6255 .getClientName())) { 6256 throw new LeaseExpiredException("Lease mismatch: " + block + 6257 " is accessed by a non lease holder " + clientName); 6258 } 6259 6260 return file; 6261 } 6262 6263 /** 6264 * Client is reporting some bad block locations. 6265 */ 6266 void reportBadBlocks(LocatedBlock[] blocks) throws IOException { 6267 checkOperation(OperationCategory.WRITE); 6268 writeLock(); 6269 try { 6270 checkOperation(OperationCategory.WRITE); 6271 for (int i = 0; i < blocks.length; i++) { 6272 ExtendedBlock blk = blocks[i].getBlock(); 6273 DatanodeInfo[] nodes = blocks[i].getLocations(); 6274 String[] storageIDs = blocks[i].getStorageIDs(); 6275 for (int j = 0; j < nodes.length; j++) { 6276 NameNode.stateChangeLog.info("*DIR* reportBadBlocks for block: {} on" 6277 + " datanode: {}", blk, nodes[j].getXferAddr()); 6278 blockManager.findAndMarkBlockAsCorrupt(blk, nodes[j], 6279 storageIDs == null ? null: storageIDs[j], 6280 "client machine reported it"); 6281 } 6282 } 6283 } finally { 6284 writeUnlock(); 6285 } 6286 } 6287 6288 /** 6289 * Get a new generation stamp together with an access token for 6290 * a block under construction 6291 * 6292 * This method is called for recovering a failed pipeline or setting up 6293 * a pipeline to append to a block. 6294 * 6295 * @param block a block 6296 * @param clientName the name of a client 6297 * @return a located block with a new generation stamp and an access token 6298 * @throws IOException if any error occurs 6299 */ 6300 LocatedBlock updateBlockForPipeline(ExtendedBlock block, 6301 String clientName) throws IOException { 6302 LocatedBlock locatedBlock; 6303 checkOperation(OperationCategory.WRITE); 6304 writeLock(); 6305 try { 6306 checkOperation(OperationCategory.WRITE); 6307 6308 // check vadility of parameters 6309 checkUCBlock(block, clientName); 6310 6311 // get a new generation stamp and an access token 6312 block.setGenerationStamp(nextGenerationStamp(blockIdManager.isLegacyBlock(block.getLocalBlock()))); 6313 locatedBlock = new LocatedBlock(block, new DatanodeInfo[0]); 6314 blockManager.setBlockToken(locatedBlock, AccessMode.WRITE); 6315 } finally { 6316 writeUnlock(); 6317 } 6318 // Ensure we record the new generation stamp 6319 getEditLog().logSync(); 6320 return locatedBlock; 6321 } 6322 6323 /** 6324 * Update a pipeline for a block under construction 6325 * 6326 * @param clientName the name of the client 6327 * @param oldBlock and old block 6328 * @param newBlock a new block with a new generation stamp and length 6329 * @param newNodes datanodes in the pipeline 6330 * @throws IOException if any error occurs 6331 */ 6332 void updatePipeline( 6333 String clientName, ExtendedBlock oldBlock, ExtendedBlock newBlock, 6334 DatanodeID[] newNodes, String[] newStorageIDs, boolean logRetryCache) 6335 throws IOException { 6336 LOG.info("updatePipeline(" + oldBlock.getLocalBlock() 6337 + ", newGS=" + newBlock.getGenerationStamp() 6338 + ", newLength=" + newBlock.getNumBytes() 6339 + ", newNodes=" + Arrays.asList(newNodes) 6340 + ", client=" + clientName 6341 + ")"); 6342 waitForLoadingFSImage(); 6343 writeLock(); 6344 try { 6345 checkOperation(OperationCategory.WRITE); 6346 checkNameNodeSafeMode("Pipeline not updated"); 6347 assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and " 6348 + oldBlock + " has different block identifier"; 6349 updatePipelineInternal(clientName, oldBlock, newBlock, newNodes, 6350 newStorageIDs, logRetryCache); 6351 } finally { 6352 writeUnlock(); 6353 } 6354 getEditLog().logSync(); 6355 LOG.info("updatePipeline(" + oldBlock.getLocalBlock() + " => " 6356 + newBlock.getLocalBlock() + ") success"); 6357 } 6358 6359 private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock, 6360 ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs, 6361 boolean logRetryCache) 6362 throws IOException { 6363 assert hasWriteLock(); 6364 // check the vadility of the block and lease holder name 6365 final INodeFile pendingFile = checkUCBlock(oldBlock, clientName); 6366 final String src = pendingFile.getFullPathName(); 6367 final BlockInfoContiguousUnderConstruction blockinfo 6368 = (BlockInfoContiguousUnderConstruction)pendingFile.getLastBlock(); 6369 6370 // check new GS & length: this is not expected 6371 if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() || 6372 newBlock.getNumBytes() < blockinfo.getNumBytes()) { 6373 String msg = "Update " + oldBlock + " (len = " + 6374 blockinfo.getNumBytes() + ") to an older state: " + newBlock + 6375 " (len = " + newBlock.getNumBytes() +")"; 6376 LOG.warn(msg); 6377 throw new IOException(msg); 6378 } 6379 6380 // Update old block with the new generation stamp and new length 6381 blockinfo.setNumBytes(newBlock.getNumBytes()); 6382 blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp()); 6383 6384 // find the DatanodeDescriptor objects 6385 final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager() 6386 .getDatanodeStorageInfos(newNodes, newStorageIDs, 6387 "src=%s, oldBlock=%s, newBlock=%s, clientName=%s", 6388 src, oldBlock, newBlock, clientName); 6389 blockinfo.setExpectedLocations(storages); 6390 6391 persistBlocks(src, pendingFile, logRetryCache); 6392 } 6393 6394 // rename was successful. If any part of the renamed subtree had 6395 // files that were being written to, update with new filename. 6396 void unprotectedChangeLease(String src, String dst) { 6397 assert hasWriteLock(); 6398 leaseManager.changeLease(src, dst); 6399 } 6400 6401 /** 6402 * Serializes leases. 6403 */ 6404 void saveFilesUnderConstruction(DataOutputStream out, 6405 Map<Long, INodeFile> snapshotUCMap) throws IOException { 6406 // This is run by an inferior thread of saveNamespace, which holds a read 6407 // lock on our behalf. If we took the read lock here, we could block 6408 // for fairness if a writer is waiting on the lock. 6409 synchronized (leaseManager) { 6410 Map<String, INodeFile> nodes = leaseManager.getINodesUnderConstruction(); 6411 for (Map.Entry<String, INodeFile> entry : nodes.entrySet()) { 6412 // TODO: for HDFS-5428, because of rename operations, some 6413 // under-construction files that are 6414 // in the current fs directory can also be captured in the 6415 // snapshotUCMap. We should remove them from the snapshotUCMap. 6416 snapshotUCMap.remove(entry.getValue().getId()); 6417 } 6418 6419 out.writeInt(nodes.size() + snapshotUCMap.size()); // write the size 6420 for (Map.Entry<String, INodeFile> entry : nodes.entrySet()) { 6421 FSImageSerialization.writeINodeUnderConstruction( 6422 out, entry.getValue(), entry.getKey()); 6423 } 6424 for (Map.Entry<Long, INodeFile> entry : snapshotUCMap.entrySet()) { 6425 // for those snapshot INodeFileUC, we use "/.reserved/.inodes/<inodeid>" 6426 // as their paths 6427 StringBuilder b = new StringBuilder(); 6428 b.append(FSDirectory.DOT_RESERVED_PATH_PREFIX) 6429 .append(Path.SEPARATOR).append(FSDirectory.DOT_INODES_STRING) 6430 .append(Path.SEPARATOR).append(entry.getValue().getId()); 6431 FSImageSerialization.writeINodeUnderConstruction( 6432 out, entry.getValue(), b.toString()); 6433 } 6434 } 6435 } 6436 6437 /** 6438 * @return all the under-construction files in the lease map 6439 */ 6440 Map<String, INodeFile> getFilesUnderConstruction() { 6441 synchronized (leaseManager) { 6442 return leaseManager.getINodesUnderConstruction(); 6443 } 6444 } 6445 6446 /** 6447 * Register a Backup name-node, verifying that it belongs 6448 * to the correct namespace, and adding it to the set of 6449 * active journals if necessary. 6450 * 6451 * @param bnReg registration of the new BackupNode 6452 * @param nnReg registration of this NameNode 6453 * @throws IOException if the namespace IDs do not match 6454 */ 6455 void registerBackupNode(NamenodeRegistration bnReg, 6456 NamenodeRegistration nnReg) throws IOException { 6457 writeLock(); 6458 try { 6459 if(getFSImage().getStorage().getNamespaceID() 6460 != bnReg.getNamespaceID()) 6461 throw new IOException("Incompatible namespaceIDs: " 6462 + " Namenode namespaceID = " 6463 + getFSImage().getStorage().getNamespaceID() + "; " 6464 + bnReg.getRole() + 6465 " node namespaceID = " + bnReg.getNamespaceID()); 6466 if (bnReg.getRole() == NamenodeRole.BACKUP) { 6467 getFSImage().getEditLog().registerBackupNode( 6468 bnReg, nnReg); 6469 } 6470 } finally { 6471 writeUnlock(); 6472 } 6473 } 6474 6475 /** 6476 * Release (unregister) backup node. 6477 * <p> 6478 * Find and remove the backup stream corresponding to the node. 6479 * @throws IOException 6480 */ 6481 void releaseBackupNode(NamenodeRegistration registration) 6482 throws IOException { 6483 checkOperation(OperationCategory.WRITE); 6484 writeLock(); 6485 try { 6486 checkOperation(OperationCategory.WRITE); 6487 if(getFSImage().getStorage().getNamespaceID() 6488 != registration.getNamespaceID()) 6489 throw new IOException("Incompatible namespaceIDs: " 6490 + " Namenode namespaceID = " 6491 + getFSImage().getStorage().getNamespaceID() + "; " 6492 + registration.getRole() + 6493 " node namespaceID = " + registration.getNamespaceID()); 6494 getEditLog().releaseBackupStream(registration); 6495 } finally { 6496 writeUnlock(); 6497 } 6498 } 6499 6500 static class CorruptFileBlockInfo { 6501 final String path; 6502 final Block block; 6503 6504 public CorruptFileBlockInfo(String p, Block b) { 6505 path = p; 6506 block = b; 6507 } 6508 6509 @Override 6510 public String toString() { 6511 return block.getBlockName() + "\t" + path; 6512 } 6513 } 6514 /** 6515 * @param path Restrict corrupt files to this portion of namespace. 6516 * @param cookieTab Support for continuation; cookieTab tells where 6517 * to start from 6518 * @return a list in which each entry describes a corrupt file/block 6519 * @throws IOException 6520 */ 6521 Collection<CorruptFileBlockInfo> listCorruptFileBlocks(String path, 6522 String[] cookieTab) throws IOException { 6523 checkSuperuserPrivilege(); 6524 checkOperation(OperationCategory.READ); 6525 6526 int count = 0; 6527 ArrayList<CorruptFileBlockInfo> corruptFiles = 6528 new ArrayList<CorruptFileBlockInfo>(); 6529 if (cookieTab == null) { 6530 cookieTab = new String[] { null }; 6531 } 6532 6533 // Do a quick check if there are any corrupt files without taking the lock 6534 if (blockManager.getMissingBlocksCount() == 0) { 6535 if (cookieTab[0] == null) { 6536 cookieTab[0] = String.valueOf(getIntCookie(cookieTab[0])); 6537 } 6538 if (LOG.isDebugEnabled()) { 6539 LOG.debug("there are no corrupt file blocks."); 6540 } 6541 return corruptFiles; 6542 } 6543 6544 readLock(); 6545 try { 6546 checkOperation(OperationCategory.READ); 6547 if (!isPopulatingReplQueues()) { 6548 throw new IOException("Cannot run listCorruptFileBlocks because " + 6549 "replication queues have not been initialized."); 6550 } 6551 // print a limited # of corrupt files per call 6552 6553 final Iterator<Block> blkIterator = blockManager.getCorruptReplicaBlockIterator(); 6554 6555 int skip = getIntCookie(cookieTab[0]); 6556 for (int i = 0; i < skip && blkIterator.hasNext(); i++) { 6557 blkIterator.next(); 6558 } 6559 6560 while (blkIterator.hasNext()) { 6561 Block blk = blkIterator.next(); 6562 final INode inode = (INode)blockManager.getBlockCollection(blk); 6563 skip++; 6564 if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) { 6565 String src = FSDirectory.getFullPathName(inode); 6566 if (src.startsWith(path)){ 6567 corruptFiles.add(new CorruptFileBlockInfo(src, blk)); 6568 count++; 6569 if (count >= DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED) 6570 break; 6571 } 6572 } 6573 } 6574 cookieTab[0] = String.valueOf(skip); 6575 if (LOG.isDebugEnabled()) { 6576 LOG.debug("list corrupt file blocks returned: " + count); 6577 } 6578 return corruptFiles; 6579 } finally { 6580 readUnlock(); 6581 } 6582 } 6583 6584 /** 6585 * Convert string cookie to integer. 6586 */ 6587 private static int getIntCookie(String cookie){ 6588 int c; 6589 if(cookie == null){ 6590 c = 0; 6591 } else { 6592 try{ 6593 c = Integer.parseInt(cookie); 6594 }catch (NumberFormatException e) { 6595 c = 0; 6596 } 6597 } 6598 c = Math.max(0, c); 6599 return c; 6600 } 6601 6602 /** 6603 * Create delegation token secret manager 6604 */ 6605 private DelegationTokenSecretManager createDelegationTokenSecretManager( 6606 Configuration conf) { 6607 return new DelegationTokenSecretManager(conf.getLong( 6608 DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_KEY, 6609 DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT), 6610 conf.getLong(DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, 6611 DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT), 6612 conf.getLong(DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, 6613 DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT), 6614 DELEGATION_TOKEN_REMOVER_SCAN_INTERVAL, 6615 conf.getBoolean(DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_KEY, 6616 DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_DEFAULT), 6617 this); 6618 } 6619 6620 /** 6621 * Returns the DelegationTokenSecretManager instance in the namesystem. 6622 * @return delegation token secret manager object 6623 */ 6624 DelegationTokenSecretManager getDelegationTokenSecretManager() { 6625 return dtSecretManager; 6626 } 6627 6628 /** 6629 * @param renewer Renewer information 6630 * @return delegation toek 6631 * @throws IOException on error 6632 */ 6633 Token<DelegationTokenIdentifier> getDelegationToken(Text renewer) 6634 throws IOException { 6635 Token<DelegationTokenIdentifier> token; 6636 checkOperation(OperationCategory.WRITE); 6637 writeLock(); 6638 try { 6639 checkOperation(OperationCategory.WRITE); 6640 checkNameNodeSafeMode("Cannot issue delegation token"); 6641 if (!isAllowedDelegationTokenOp()) { 6642 throw new IOException( 6643 "Delegation Token can be issued only with kerberos or web authentication"); 6644 } 6645 if (dtSecretManager == null || !dtSecretManager.isRunning()) { 6646 LOG.warn("trying to get DT with no secret manager running"); 6647 return null; 6648 } 6649 6650 UserGroupInformation ugi = getRemoteUser(); 6651 String user = ugi.getUserName(); 6652 Text owner = new Text(user); 6653 Text realUser = null; 6654 if (ugi.getRealUser() != null) { 6655 realUser = new Text(ugi.getRealUser().getUserName()); 6656 } 6657 DelegationTokenIdentifier dtId = new DelegationTokenIdentifier(owner, 6658 renewer, realUser); 6659 token = new Token<DelegationTokenIdentifier>( 6660 dtId, dtSecretManager); 6661 long expiryTime = dtSecretManager.getTokenExpiryTime(dtId); 6662 getEditLog().logGetDelegationToken(dtId, expiryTime); 6663 } finally { 6664 writeUnlock(); 6665 } 6666 getEditLog().logSync(); 6667 return token; 6668 } 6669 6670 /** 6671 * 6672 * @param token token to renew 6673 * @return new expiryTime of the token 6674 * @throws InvalidToken if {@code token} is invalid 6675 * @throws IOException on other errors 6676 */ 6677 long renewDelegationToken(Token<DelegationTokenIdentifier> token) 6678 throws InvalidToken, IOException { 6679 long expiryTime; 6680 checkOperation(OperationCategory.WRITE); 6681 writeLock(); 6682 try { 6683 checkOperation(OperationCategory.WRITE); 6684 6685 checkNameNodeSafeMode("Cannot renew delegation token"); 6686 if (!isAllowedDelegationTokenOp()) { 6687 throw new IOException( 6688 "Delegation Token can be renewed only with kerberos or web authentication"); 6689 } 6690 String renewer = getRemoteUser().getShortUserName(); 6691 expiryTime = dtSecretManager.renewToken(token, renewer); 6692 DelegationTokenIdentifier id = new DelegationTokenIdentifier(); 6693 ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier()); 6694 DataInputStream in = new DataInputStream(buf); 6695 id.readFields(in); 6696 getEditLog().logRenewDelegationToken(id, expiryTime); 6697 } finally { 6698 writeUnlock(); 6699 } 6700 getEditLog().logSync(); 6701 return expiryTime; 6702 } 6703 6704 /** 6705 * 6706 * @param token token to cancel 6707 * @throws IOException on error 6708 */ 6709 void cancelDelegationToken(Token<DelegationTokenIdentifier> token) 6710 throws IOException { 6711 checkOperation(OperationCategory.WRITE); 6712 writeLock(); 6713 try { 6714 checkOperation(OperationCategory.WRITE); 6715 6716 checkNameNodeSafeMode("Cannot cancel delegation token"); 6717 String canceller = getRemoteUser().getUserName(); 6718 DelegationTokenIdentifier id = dtSecretManager 6719 .cancelToken(token, canceller); 6720 getEditLog().logCancelDelegationToken(id); 6721 } finally { 6722 writeUnlock(); 6723 } 6724 getEditLog().logSync(); 6725 } 6726 6727 /** 6728 * @param out save state of the secret manager 6729 * @param sdPath String storage directory path 6730 */ 6731 void saveSecretManagerStateCompat(DataOutputStream out, String sdPath) 6732 throws IOException { 6733 dtSecretManager.saveSecretManagerStateCompat(out, sdPath); 6734 } 6735 6736 SecretManagerState saveSecretManagerState() { 6737 return dtSecretManager.saveSecretManagerState(); 6738 } 6739 6740 /** 6741 * @param in load the state of secret manager from input stream 6742 */ 6743 void loadSecretManagerStateCompat(DataInput in) throws IOException { 6744 dtSecretManager.loadSecretManagerStateCompat(in); 6745 } 6746 6747 void loadSecretManagerState(SecretManagerSection s, 6748 List<SecretManagerSection.DelegationKey> keys, 6749 List<SecretManagerSection.PersistToken> tokens) throws IOException { 6750 dtSecretManager.loadSecretManagerState(new SecretManagerState(s, keys, tokens)); 6751 } 6752 6753 /** 6754 * Log the updateMasterKey operation to edit logs 6755 * 6756 * @param key new delegation key. 6757 */ 6758 public void logUpdateMasterKey(DelegationKey key) { 6759 6760 assert !isInSafeMode() : 6761 "this should never be called while in safemode, since we stop " + 6762 "the DT manager before entering safemode!"; 6763 // No need to hold FSN lock since we don't access any internal 6764 // structures, and this is stopped before the FSN shuts itself 6765 // down, etc. 6766 getEditLog().logUpdateMasterKey(key); 6767 getEditLog().logSync(); 6768 } 6769 6770 /** 6771 * Log the cancellation of expired tokens to edit logs 6772 * 6773 * @param id token identifier to cancel 6774 */ 6775 public void logExpireDelegationToken(DelegationTokenIdentifier id) { 6776 assert !isInSafeMode() : 6777 "this should never be called while in safemode, since we stop " + 6778 "the DT manager before entering safemode!"; 6779 // No need to hold FSN lock since we don't access any internal 6780 // structures, and this is stopped before the FSN shuts itself 6781 // down, etc. 6782 getEditLog().logCancelDelegationToken(id); 6783 } 6784 6785 private void logReassignLease(String leaseHolder, String src, 6786 String newHolder) { 6787 assert hasWriteLock(); 6788 getEditLog().logReassignLease(leaseHolder, src, newHolder); 6789 } 6790 6791 /** 6792 * 6793 * @return true if delegation token operation is allowed 6794 */ 6795 private boolean isAllowedDelegationTokenOp() throws IOException { 6796 AuthenticationMethod authMethod = getConnectionAuthenticationMethod(); 6797 if (UserGroupInformation.isSecurityEnabled() 6798 && (authMethod != AuthenticationMethod.KERBEROS) 6799 && (authMethod != AuthenticationMethod.KERBEROS_SSL) 6800 && (authMethod != AuthenticationMethod.CERTIFICATE)) { 6801 return false; 6802 } 6803 return true; 6804 } 6805 6806 /** 6807 * Returns authentication method used to establish the connection 6808 * @return AuthenticationMethod used to establish connection 6809 * @throws IOException 6810 */ 6811 private AuthenticationMethod getConnectionAuthenticationMethod() 6812 throws IOException { 6813 UserGroupInformation ugi = getRemoteUser(); 6814 AuthenticationMethod authMethod = ugi.getAuthenticationMethod(); 6815 if (authMethod == AuthenticationMethod.PROXY) { 6816 authMethod = ugi.getRealUser().getAuthenticationMethod(); 6817 } 6818 return authMethod; 6819 } 6820 6821 /** 6822 * Client invoked methods are invoked over RPC and will be in 6823 * RPC call context even if the client exits. 6824 */ 6825 boolean isExternalInvocation() { 6826 return Server.isRpcInvocation() || NamenodeWebHdfsMethods.isWebHdfsInvocation(); 6827 } 6828 6829 private static InetAddress getRemoteIp() { 6830 InetAddress ip = Server.getRemoteIp(); 6831 if (ip != null) { 6832 return ip; 6833 } 6834 return NamenodeWebHdfsMethods.getRemoteIp(); 6835 } 6836 6837 // optimize ugi lookup for RPC operations to avoid a trip through 6838 // UGI.getCurrentUser which is synch'ed 6839 private static UserGroupInformation getRemoteUser() throws IOException { 6840 return NameNode.getRemoteUser(); 6841 } 6842 6843 /** 6844 * Log fsck event in the audit log 6845 */ 6846 void logFsckEvent(String src, InetAddress remoteAddress) throws IOException { 6847 if (isAuditEnabled()) { 6848 logAuditEvent(true, getRemoteUser(), 6849 remoteAddress, 6850 "fsck", src, null, null); 6851 } 6852 } 6853 /** 6854 * Register NameNodeMXBean 6855 */ 6856 private void registerMXBean() { 6857 mxbeanName = MBeans.register("NameNode", "NameNodeInfo", this); 6858 } 6859 6860 /** 6861 * Class representing Namenode information for JMX interfaces 6862 */ 6863 @Override // NameNodeMXBean 6864 public String getVersion() { 6865 return VersionInfo.getVersion() + ", r" + VersionInfo.getRevision(); 6866 } 6867 6868 @Override // NameNodeMXBean 6869 public long getUsed() { 6870 return this.getCapacityUsed(); 6871 } 6872 6873 @Override // NameNodeMXBean 6874 public long getFree() { 6875 return this.getCapacityRemaining(); 6876 } 6877 6878 @Override // NameNodeMXBean 6879 public long getTotal() { 6880 return this.getCapacityTotal(); 6881 } 6882 6883 @Override // NameNodeMXBean 6884 public String getSafemode() { 6885 if (!this.isInSafeMode()) 6886 return ""; 6887 return "Safe mode is ON. " + this.getSafeModeTip(); 6888 } 6889 6890 @Override // NameNodeMXBean 6891 public boolean isUpgradeFinalized() { 6892 return this.getFSImage().isUpgradeFinalized(); 6893 } 6894 6895 @Override // NameNodeMXBean 6896 public long getNonDfsUsedSpace() { 6897 return datanodeStatistics.getCapacityUsedNonDFS(); 6898 } 6899 6900 @Override // NameNodeMXBean 6901 public float getPercentUsed() { 6902 return datanodeStatistics.getCapacityUsedPercent(); 6903 } 6904 6905 @Override // NameNodeMXBean 6906 public long getBlockPoolUsedSpace() { 6907 return datanodeStatistics.getBlockPoolUsed(); 6908 } 6909 6910 @Override // NameNodeMXBean 6911 public float getPercentBlockPoolUsed() { 6912 return datanodeStatistics.getPercentBlockPoolUsed(); 6913 } 6914 6915 @Override // NameNodeMXBean 6916 public float getPercentRemaining() { 6917 return datanodeStatistics.getCapacityRemainingPercent(); 6918 } 6919 6920 @Override // NameNodeMXBean 6921 public long getCacheCapacity() { 6922 return datanodeStatistics.getCacheCapacity(); 6923 } 6924 6925 @Override // NameNodeMXBean 6926 public long getCacheUsed() { 6927 return datanodeStatistics.getCacheUsed(); 6928 } 6929 6930 @Override // NameNodeMXBean 6931 public long getTotalBlocks() { 6932 return getBlocksTotal(); 6933 } 6934 6935 @Override // NameNodeMXBean 6936 @Metric 6937 public long getTotalFiles() { 6938 return getFilesTotal(); 6939 } 6940 6941 @Override // NameNodeMXBean 6942 public long getNumberOfMissingBlocks() { 6943 return getMissingBlocksCount(); 6944 } 6945 6946 @Override // NameNodeMXBean 6947 public long getNumberOfMissingBlocksWithReplicationFactorOne() { 6948 return getMissingReplOneBlocksCount(); 6949 } 6950 6951 @Override // NameNodeMXBean 6952 public int getThreads() { 6953 return ManagementFactory.getThreadMXBean().getThreadCount(); 6954 } 6955 6956 /** 6957 * Returned information is a JSON representation of map with host name as the 6958 * key and value is a map of live node attribute keys to its values 6959 */ 6960 @Override // NameNodeMXBean 6961 public String getLiveNodes() { 6962 final Map<String, Map<String,Object>> info = 6963 new HashMap<String, Map<String,Object>>(); 6964 final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); 6965 blockManager.getDatanodeManager().fetchDatanodes(live, null, true); 6966 for (DatanodeDescriptor node : live) { 6967 ImmutableMap.Builder<String, Object> innerinfo = 6968 ImmutableMap.<String,Object>builder(); 6969 innerinfo 6970 .put("infoAddr", node.getInfoAddr()) 6971 .put("infoSecureAddr", node.getInfoSecureAddr()) 6972 .put("xferaddr", node.getXferAddr()) 6973 .put("lastContact", getLastContact(node)) 6974 .put("usedSpace", getDfsUsed(node)) 6975 .put("adminState", node.getAdminState().toString()) 6976 .put("nonDfsUsedSpace", node.getNonDfsUsed()) 6977 .put("capacity", node.getCapacity()) 6978 .put("numBlocks", node.numBlocks()) 6979 .put("version", node.getSoftwareVersion()) 6980 .put("used", node.getDfsUsed()) 6981 .put("remaining", node.getRemaining()) 6982 .put("blockScheduled", node.getBlocksScheduled()) 6983 .put("blockPoolUsed", node.getBlockPoolUsed()) 6984 .put("blockPoolUsedPercent", node.getBlockPoolUsedPercent()) 6985 .put("volfails", node.getVolumeFailures()); 6986 VolumeFailureSummary volumeFailureSummary = node.getVolumeFailureSummary(); 6987 if (volumeFailureSummary != null) { 6988 innerinfo 6989 .put("failedStorageLocations", 6990 volumeFailureSummary.getFailedStorageLocations()) 6991 .put("lastVolumeFailureDate", 6992 volumeFailureSummary.getLastVolumeFailureDate()) 6993 .put("estimatedCapacityLostTotal", 6994 volumeFailureSummary.getEstimatedCapacityLostTotal()); 6995 } 6996 info.put(node.getHostName() + ":" + node.getXferPort(), innerinfo.build()); 6997 } 6998 return JSON.toString(info); 6999 } 7000 7001 /** 7002 * Returned information is a JSON representation of map with host name as the 7003 * key and value is a map of dead node attribute keys to its values 7004 */ 7005 @Override // NameNodeMXBean 7006 public String getDeadNodes() { 7007 final Map<String, Map<String, Object>> info = 7008 new HashMap<String, Map<String, Object>>(); 7009 final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>(); 7010 blockManager.getDatanodeManager().fetchDatanodes(null, dead, true); 7011 for (DatanodeDescriptor node : dead) { 7012 Map<String, Object> innerinfo = ImmutableMap.<String, Object>builder() 7013 .put("lastContact", getLastContact(node)) 7014 .put("decommissioned", node.isDecommissioned()) 7015 .put("xferaddr", node.getXferAddr()) 7016 .build(); 7017 info.put(node.getHostName() + ":" + node.getXferPort(), innerinfo); 7018 } 7019 return JSON.toString(info); 7020 } 7021 7022 /** 7023 * Returned information is a JSON representation of map with host name as the 7024 * key and value is a map of decommissioning node attribute keys to its 7025 * values 7026 */ 7027 @Override // NameNodeMXBean 7028 public String getDecomNodes() { 7029 final Map<String, Map<String, Object>> info = 7030 new HashMap<String, Map<String, Object>>(); 7031 final List<DatanodeDescriptor> decomNodeList = blockManager.getDatanodeManager( 7032 ).getDecommissioningNodes(); 7033 for (DatanodeDescriptor node : decomNodeList) { 7034 Map<String, Object> innerinfo = ImmutableMap 7035 .<String, Object> builder() 7036 .put("xferaddr", node.getXferAddr()) 7037 .put("underReplicatedBlocks", 7038 node.decommissioningStatus.getUnderReplicatedBlocks()) 7039 .put("decommissionOnlyReplicas", 7040 node.decommissioningStatus.getDecommissionOnlyReplicas()) 7041 .put("underReplicateInOpenFiles", 7042 node.decommissioningStatus.getUnderReplicatedInOpenFiles()) 7043 .build(); 7044 info.put(node.getHostName() + ":" + node.getXferPort(), innerinfo); 7045 } 7046 return JSON.toString(info); 7047 } 7048 7049 private long getLastContact(DatanodeDescriptor alivenode) { 7050 return (monotonicNow() - alivenode.getLastUpdateMonotonic())/1000; 7051 } 7052 7053 private long getDfsUsed(DatanodeDescriptor alivenode) { 7054 return alivenode.getDfsUsed(); 7055 } 7056 7057 @Override // NameNodeMXBean 7058 public String getClusterId() { 7059 return getFSImage().getStorage().getClusterID(); 7060 } 7061 7062 @Override // NameNodeMXBean 7063 public String getBlockPoolId() { 7064 return blockPoolId; 7065 } 7066 7067 @Override // NameNodeMXBean 7068 public String getNameDirStatuses() { 7069 Map<String, Map<File, StorageDirType>> statusMap = 7070 new HashMap<String, Map<File, StorageDirType>>(); 7071 7072 Map<File, StorageDirType> activeDirs = new HashMap<File, StorageDirType>(); 7073 for (Iterator<StorageDirectory> it 7074 = getFSImage().getStorage().dirIterator(); it.hasNext();) { 7075 StorageDirectory st = it.next(); 7076 activeDirs.put(st.getRoot(), st.getStorageDirType()); 7077 } 7078 statusMap.put("active", activeDirs); 7079 7080 List<Storage.StorageDirectory> removedStorageDirs 7081 = getFSImage().getStorage().getRemovedStorageDirs(); 7082 Map<File, StorageDirType> failedDirs = new HashMap<File, StorageDirType>(); 7083 for (StorageDirectory st : removedStorageDirs) { 7084 failedDirs.put(st.getRoot(), st.getStorageDirType()); 7085 } 7086 statusMap.put("failed", failedDirs); 7087 7088 return JSON.toString(statusMap); 7089 } 7090 7091 @Override // NameNodeMXBean 7092 public String getNodeUsage() { 7093 float median = 0; 7094 float max = 0; 7095 float min = 0; 7096 float dev = 0; 7097 7098 final Map<String, Map<String,Object>> info = 7099 new HashMap<String, Map<String,Object>>(); 7100 final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); 7101 blockManager.getDatanodeManager().fetchDatanodes(live, null, true); 7102 7103 if (live.size() > 0) { 7104 float totalDfsUsed = 0; 7105 float[] usages = new float[live.size()]; 7106 int i = 0; 7107 for (DatanodeDescriptor dn : live) { 7108 usages[i++] = dn.getDfsUsedPercent(); 7109 totalDfsUsed += dn.getDfsUsedPercent(); 7110 } 7111 totalDfsUsed /= live.size(); 7112 Arrays.sort(usages); 7113 median = usages[usages.length / 2]; 7114 max = usages[usages.length - 1]; 7115 min = usages[0]; 7116 7117 for (i = 0; i < usages.length; i++) { 7118 dev += (usages[i] - totalDfsUsed) * (usages[i] - totalDfsUsed); 7119 } 7120 dev = (float) Math.sqrt(dev / usages.length); 7121 } 7122 7123 final Map<String, Object> innerInfo = new HashMap<String, Object>(); 7124 innerInfo.put("min", StringUtils.format("%.2f%%", min)); 7125 innerInfo.put("median", StringUtils.format("%.2f%%", median)); 7126 innerInfo.put("max", StringUtils.format("%.2f%%", max)); 7127 innerInfo.put("stdDev", StringUtils.format("%.2f%%", dev)); 7128 info.put("nodeUsage", innerInfo); 7129 7130 return JSON.toString(info); 7131 } 7132 7133 @Override // NameNodeMXBean 7134 public String getNameJournalStatus() { 7135 List<Map<String, String>> jasList = new ArrayList<Map<String, String>>(); 7136 FSEditLog log = getFSImage().getEditLog(); 7137 if (log != null) { 7138 boolean openForWrite = log.isOpenForWrite(); 7139 for (JournalAndStream jas : log.getJournals()) { 7140 final Map<String, String> jasMap = new HashMap<String, String>(); 7141 String manager = jas.getManager().toString(); 7142 7143 jasMap.put("required", String.valueOf(jas.isRequired())); 7144 jasMap.put("disabled", String.valueOf(jas.isDisabled())); 7145 jasMap.put("manager", manager); 7146 7147 if (jas.isDisabled()) { 7148 jasMap.put("stream", "Failed"); 7149 } else if (openForWrite) { 7150 EditLogOutputStream elos = jas.getCurrentStream(); 7151 if (elos != null) { 7152 jasMap.put("stream", elos.generateReport()); 7153 } else { 7154 jasMap.put("stream", "not currently writing"); 7155 } 7156 } else { 7157 jasMap.put("stream", "open for read"); 7158 } 7159 jasList.add(jasMap); 7160 } 7161 } 7162 return JSON.toString(jasList); 7163 } 7164 7165 @Override // NameNodeMxBean 7166 public String getJournalTransactionInfo() { 7167 Map<String, String> txnIdMap = new HashMap<String, String>(); 7168 txnIdMap.put("LastAppliedOrWrittenTxId", 7169 Long.toString(this.getFSImage().getLastAppliedOrWrittenTxId())); 7170 txnIdMap.put("MostRecentCheckpointTxId", 7171 Long.toString(this.getFSImage().getMostRecentCheckpointTxId())); 7172 return JSON.toString(txnIdMap); 7173 } 7174 7175 @Override // NameNodeMXBean 7176 public String getNNStarted() { 7177 return getStartTime().toString(); 7178 } 7179 7180 @Override // NameNodeMXBean 7181 public String getCompileInfo() { 7182 return VersionInfo.getDate() + " by " + VersionInfo.getUser() + 7183 " from " + VersionInfo.getBranch(); 7184 } 7185 7186 /** @return the block manager. */ 7187 public BlockManager getBlockManager() { 7188 return blockManager; 7189 } 7190 7191 public BlockIdManager getBlockIdManager() { 7192 return blockIdManager; 7193 } 7194 7195 /** @return the FSDirectory. */ 7196 public FSDirectory getFSDirectory() { 7197 return dir; 7198 } 7199 /** Set the FSDirectory. */ 7200 @VisibleForTesting 7201 public void setFSDirectory(FSDirectory dir) { 7202 this.dir = dir; 7203 } 7204 /** @return the cache manager. */ 7205 public CacheManager getCacheManager() { 7206 return cacheManager; 7207 } 7208 7209 @Override // NameNodeMXBean 7210 public String getCorruptFiles() { 7211 List<String> list = new ArrayList<String>(); 7212 Collection<FSNamesystem.CorruptFileBlockInfo> corruptFileBlocks; 7213 try { 7214 corruptFileBlocks = listCorruptFileBlocks("/", null); 7215 int corruptFileCount = corruptFileBlocks.size(); 7216 if (corruptFileCount != 0) { 7217 for (FSNamesystem.CorruptFileBlockInfo c : corruptFileBlocks) { 7218 list.add(c.toString()); 7219 } 7220 } 7221 } catch (IOException e) { 7222 LOG.warn("Get corrupt file blocks returned error: " + e.getMessage()); 7223 } 7224 return JSON.toString(list); 7225 } 7226 7227 @Override //NameNodeMXBean 7228 public int getDistinctVersionCount() { 7229 return blockManager.getDatanodeManager().getDatanodesSoftwareVersions() 7230 .size(); 7231 } 7232 7233 @Override //NameNodeMXBean 7234 public Map<String, Integer> getDistinctVersions() { 7235 return blockManager.getDatanodeManager().getDatanodesSoftwareVersions(); 7236 } 7237 7238 @Override //NameNodeMXBean 7239 public String getSoftwareVersion() { 7240 return VersionInfo.getVersion(); 7241 } 7242 7243 /** 7244 * Verifies that the given identifier and password are valid and match. 7245 * @param identifier Token identifier. 7246 * @param password Password in the token. 7247 */ 7248 public synchronized void verifyToken(DelegationTokenIdentifier identifier, 7249 byte[] password) throws InvalidToken, RetriableException { 7250 try { 7251 getDelegationTokenSecretManager().verifyToken(identifier, password); 7252 } catch (InvalidToken it) { 7253 if (inTransitionToActive()) { 7254 throw new RetriableException(it); 7255 } 7256 throw it; 7257 } 7258 } 7259 7260 @Override 7261 public boolean isGenStampInFuture(Block block) { 7262 return blockIdManager.isGenStampInFuture(block); 7263 } 7264 7265 @VisibleForTesting 7266 public EditLogTailer getEditLogTailer() { 7267 return editLogTailer; 7268 } 7269 7270 @VisibleForTesting 7271 public void setEditLogTailerForTests(EditLogTailer tailer) { 7272 this.editLogTailer = tailer; 7273 } 7274 7275 @VisibleForTesting 7276 void setFsLockForTests(ReentrantReadWriteLock lock) { 7277 this.fsLock.coarseLock = lock; 7278 } 7279 7280 @VisibleForTesting 7281 public ReentrantReadWriteLock getFsLockForTests() { 7282 return fsLock.coarseLock; 7283 } 7284 7285 @VisibleForTesting 7286 public ReentrantLock getCpLockForTests() { 7287 return cpLock; 7288 } 7289 7290 @VisibleForTesting 7291 public SafeModeInfo getSafeModeInfoForTests() { 7292 return safeMode; 7293 } 7294 7295 @VisibleForTesting 7296 public void setNNResourceChecker(NameNodeResourceChecker nnResourceChecker) { 7297 this.nnResourceChecker = nnResourceChecker; 7298 } 7299 7300 public SnapshotManager getSnapshotManager() { 7301 return snapshotManager; 7302 } 7303 7304 /** Allow snapshot on a directory. */ 7305 void allowSnapshot(String path) throws IOException { 7306 checkOperation(OperationCategory.WRITE); 7307 boolean success = false; 7308 writeLock(); 7309 try { 7310 checkOperation(OperationCategory.WRITE); 7311 checkNameNodeSafeMode("Cannot allow snapshot for " + path); 7312 checkSuperuserPrivilege(); 7313 FSDirSnapshotOp.allowSnapshot(dir, snapshotManager, path); 7314 success = true; 7315 } finally { 7316 writeUnlock(); 7317 } 7318 getEditLog().logSync(); 7319 logAuditEvent(success, "allowSnapshot", path, null, null); 7320 } 7321 7322 /** Disallow snapshot on a directory. */ 7323 void disallowSnapshot(String path) throws IOException { 7324 checkOperation(OperationCategory.WRITE); 7325 boolean success = false; 7326 writeLock(); 7327 try { 7328 checkOperation(OperationCategory.WRITE); 7329 checkNameNodeSafeMode("Cannot disallow snapshot for " + path); 7330 checkSuperuserPrivilege(); 7331 FSDirSnapshotOp.disallowSnapshot(dir, snapshotManager, path); 7332 success = true; 7333 } finally { 7334 writeUnlock(); 7335 } 7336 getEditLog().logSync(); 7337 logAuditEvent(success, "disallowSnapshot", path, null, null); 7338 } 7339 7340 /** 7341 * Create a snapshot 7342 * @param snapshotRoot The directory path where the snapshot is taken 7343 * @param snapshotName The name of the snapshot 7344 */ 7345 String createSnapshot(String snapshotRoot, String snapshotName, 7346 boolean logRetryCache) throws IOException { 7347 String snapshotPath = null; 7348 writeLock(); 7349 try { 7350 checkOperation(OperationCategory.WRITE); 7351 checkNameNodeSafeMode("Cannot create snapshot for " + snapshotRoot); 7352 snapshotPath = FSDirSnapshotOp.createSnapshot(dir, 7353 snapshotManager, snapshotRoot, snapshotName, logRetryCache); 7354 } finally { 7355 writeUnlock(); 7356 } 7357 getEditLog().logSync(); 7358 logAuditEvent(snapshotPath != null, "createSnapshot", snapshotRoot, 7359 snapshotPath, null); 7360 return snapshotPath; 7361 } 7362 7363 /** 7364 * Rename a snapshot 7365 * @param path The directory path where the snapshot was taken 7366 * @param snapshotOldName Old snapshot name 7367 * @param snapshotNewName New snapshot name 7368 * @throws SafeModeException 7369 * @throws IOException 7370 */ 7371 void renameSnapshot( 7372 String path, String snapshotOldName, String snapshotNewName, 7373 boolean logRetryCache) throws IOException { 7374 boolean success = false; 7375 writeLock(); 7376 try { 7377 checkOperation(OperationCategory.WRITE); 7378 checkNameNodeSafeMode("Cannot rename snapshot for " + path); 7379 FSDirSnapshotOp.renameSnapshot(dir, snapshotManager, path, 7380 snapshotOldName, snapshotNewName, logRetryCache); 7381 success = true; 7382 } finally { 7383 writeUnlock(); 7384 } 7385 getEditLog().logSync(); 7386 String oldSnapshotRoot = Snapshot.getSnapshotPath(path, snapshotOldName); 7387 String newSnapshotRoot = Snapshot.getSnapshotPath(path, snapshotNewName); 7388 logAuditEvent(success, "renameSnapshot", oldSnapshotRoot, 7389 newSnapshotRoot, null); 7390 } 7391 7392 /** 7393 * Get the list of snapshottable directories that are owned 7394 * by the current user. Return all the snapshottable directories if the 7395 * current user is a super user. 7396 * @return The list of all the current snapshottable directories 7397 * @throws IOException 7398 */ 7399 public SnapshottableDirectoryStatus[] getSnapshottableDirListing() 7400 throws IOException { 7401 SnapshottableDirectoryStatus[] status = null; 7402 checkOperation(OperationCategory.READ); 7403 boolean success = false; 7404 readLock(); 7405 try { 7406 checkOperation(OperationCategory.READ); 7407 status = FSDirSnapshotOp.getSnapshottableDirListing(dir, snapshotManager); 7408 success = true; 7409 } finally { 7410 readUnlock(); 7411 } 7412 logAuditEvent(success, "listSnapshottableDirectory", null, null, null); 7413 return status; 7414 } 7415 7416 /** 7417 * Get the difference between two snapshots (or between a snapshot and the 7418 * current status) of a snapshottable directory. 7419 * 7420 * @param path The full path of the snapshottable directory. 7421 * @param fromSnapshot Name of the snapshot to calculate the diff from. Null 7422 * or empty string indicates the current tree. 7423 * @param toSnapshot Name of the snapshot to calculated the diff to. Null or 7424 * empty string indicates the current tree. 7425 * @return A report about the difference between {@code fromSnapshot} and 7426 * {@code toSnapshot}. Modified/deleted/created/renamed files and 7427 * directories belonging to the snapshottable directories are listed 7428 * and labeled as M/-/+/R respectively. 7429 * @throws IOException 7430 */ 7431 SnapshotDiffReport getSnapshotDiffReport(String path, 7432 String fromSnapshot, String toSnapshot) throws IOException { 7433 SnapshotDiffReport diffs = null; 7434 checkOperation(OperationCategory.READ); 7435 readLock(); 7436 try { 7437 checkOperation(OperationCategory.READ); 7438 diffs = FSDirSnapshotOp.getSnapshotDiffReport(dir, snapshotManager, 7439 path, fromSnapshot, toSnapshot); 7440 } finally { 7441 readUnlock(); 7442 } 7443 7444 logAuditEvent(diffs != null, "computeSnapshotDiff", null, null, null); 7445 return diffs; 7446 } 7447 7448 /** 7449 * Delete a snapshot of a snapshottable directory 7450 * @param snapshotRoot The snapshottable directory 7451 * @param snapshotName The name of the to-be-deleted snapshot 7452 * @throws SafeModeException 7453 * @throws IOException 7454 */ 7455 void deleteSnapshot(String snapshotRoot, String snapshotName, 7456 boolean logRetryCache) throws IOException { 7457 boolean success = false; 7458 writeLock(); 7459 BlocksMapUpdateInfo blocksToBeDeleted = null; 7460 try { 7461 checkOperation(OperationCategory.WRITE); 7462 checkNameNodeSafeMode("Cannot delete snapshot for " + snapshotRoot); 7463 7464 blocksToBeDeleted = FSDirSnapshotOp.deleteSnapshot(dir, snapshotManager, 7465 snapshotRoot, snapshotName, logRetryCache); 7466 success = true; 7467 } finally { 7468 writeUnlock(); 7469 } 7470 getEditLog().logSync(); 7471 7472 // Breaking the pattern as removing blocks have to happen outside of the 7473 // global lock 7474 if (blocksToBeDeleted != null) { 7475 removeBlocks(blocksToBeDeleted); 7476 } 7477 7478 String rootPath = Snapshot.getSnapshotPath(snapshotRoot, snapshotName); 7479 logAuditEvent(success, "deleteSnapshot", rootPath, null, null); 7480 } 7481 7482 /** 7483 * Remove a list of INodeDirectorySnapshottable from the SnapshotManager 7484 * @param toRemove the list of INodeDirectorySnapshottable to be removed 7485 */ 7486 void removeSnapshottableDirs(List<INodeDirectory> toRemove) { 7487 if (snapshotManager != null) { 7488 snapshotManager.removeSnapshottable(toRemove); 7489 } 7490 } 7491 7492 RollingUpgradeInfo queryRollingUpgrade() throws IOException { 7493 checkSuperuserPrivilege(); 7494 checkOperation(OperationCategory.READ); 7495 readLock(); 7496 try { 7497 if (!isRollingUpgrade()) { 7498 return null; 7499 } 7500 Preconditions.checkNotNull(rollingUpgradeInfo); 7501 boolean hasRollbackImage = this.getFSImage().hasRollbackFSImage(); 7502 rollingUpgradeInfo.setCreatedRollbackImages(hasRollbackImage); 7503 return rollingUpgradeInfo; 7504 } finally { 7505 readUnlock(); 7506 } 7507 } 7508 7509 RollingUpgradeInfo startRollingUpgrade() throws IOException { 7510 checkSuperuserPrivilege(); 7511 checkOperation(OperationCategory.WRITE); 7512 writeLock(); 7513 try { 7514 checkOperation(OperationCategory.WRITE); 7515 if (isRollingUpgrade()) { 7516 return rollingUpgradeInfo; 7517 } 7518 long startTime = now(); 7519 if (!haEnabled) { // for non-HA, we require NN to be in safemode 7520 startRollingUpgradeInternalForNonHA(startTime); 7521 } else { // for HA, NN cannot be in safemode 7522 checkNameNodeSafeMode("Failed to start rolling upgrade"); 7523 startRollingUpgradeInternal(startTime); 7524 } 7525 7526 getEditLog().logStartRollingUpgrade(rollingUpgradeInfo.getStartTime()); 7527 if (haEnabled) { 7528 // roll the edit log to make sure the standby NameNode can tail 7529 getFSImage().rollEditLog(); 7530 } 7531 } finally { 7532 writeUnlock(); 7533 } 7534 7535 getEditLog().logSync(); 7536 if (auditLog.isInfoEnabled() && isExternalInvocation()) { 7537 logAuditEvent(true, "startRollingUpgrade", null, null, null); 7538 } 7539 return rollingUpgradeInfo; 7540 } 7541 7542 /** 7543 * Update internal state to indicate that a rolling upgrade is in progress. 7544 * @param startTime rolling upgrade start time 7545 */ 7546 void startRollingUpgradeInternal(long startTime) 7547 throws IOException { 7548 checkRollingUpgrade("start rolling upgrade"); 7549 getFSImage().checkUpgrade(); 7550 setRollingUpgradeInfo(false, startTime); 7551 } 7552 7553 /** 7554 * Update internal state to indicate that a rolling upgrade is in progress for 7555 * non-HA setup. This requires the namesystem is in SafeMode and after doing a 7556 * checkpoint for rollback the namesystem will quit the safemode automatically 7557 */ 7558 private void startRollingUpgradeInternalForNonHA(long startTime) 7559 throws IOException { 7560 Preconditions.checkState(!haEnabled); 7561 if (!isInSafeMode()) { 7562 throw new IOException("Safe mode should be turned ON " 7563 + "in order to create namespace image."); 7564 } 7565 checkRollingUpgrade("start rolling upgrade"); 7566 getFSImage().checkUpgrade(); 7567 // in non-HA setup, we do an extra checkpoint to generate a rollback image 7568 getFSImage().saveNamespace(this, NameNodeFile.IMAGE_ROLLBACK, null); 7569 LOG.info("Successfully saved namespace for preparing rolling upgrade."); 7570 7571 // leave SafeMode automatically 7572 setSafeMode(SafeModeAction.SAFEMODE_LEAVE); 7573 setRollingUpgradeInfo(true, startTime); 7574 } 7575 7576 void setRollingUpgradeInfo(boolean createdRollbackImages, long startTime) { 7577 rollingUpgradeInfo = new RollingUpgradeInfo(blockPoolId, 7578 createdRollbackImages, startTime, 0L); 7579 } 7580 7581 public void setCreatedRollbackImages(boolean created) { 7582 if (rollingUpgradeInfo != null) { 7583 rollingUpgradeInfo.setCreatedRollbackImages(created); 7584 } 7585 } 7586 7587 public RollingUpgradeInfo getRollingUpgradeInfo() { 7588 return rollingUpgradeInfo; 7589 } 7590 7591 public boolean isNeedRollbackFsImage() { 7592 return needRollbackFsImage; 7593 } 7594 7595 public void setNeedRollbackFsImage(boolean needRollbackFsImage) { 7596 this.needRollbackFsImage = needRollbackFsImage; 7597 } 7598 7599 @Override // NameNodeMXBean 7600 public RollingUpgradeInfo.Bean getRollingUpgradeStatus() { 7601 if (!isRollingUpgrade()) { 7602 return null; 7603 } 7604 RollingUpgradeInfo upgradeInfo = getRollingUpgradeInfo(); 7605 if (upgradeInfo.createdRollbackImages()) { 7606 return new RollingUpgradeInfo.Bean(upgradeInfo); 7607 } 7608 readLock(); 7609 try { 7610 // check again after acquiring the read lock. 7611 upgradeInfo = getRollingUpgradeInfo(); 7612 if (upgradeInfo == null) { 7613 return null; 7614 } 7615 if (!upgradeInfo.createdRollbackImages()) { 7616 boolean hasRollbackImage = this.getFSImage().hasRollbackFSImage(); 7617 upgradeInfo.setCreatedRollbackImages(hasRollbackImage); 7618 } 7619 } catch (IOException ioe) { 7620 LOG.warn("Encountered exception setting Rollback Image", ioe); 7621 } finally { 7622 readUnlock(); 7623 } 7624 return new RollingUpgradeInfo.Bean(upgradeInfo); 7625 } 7626 7627 /** Is rolling upgrade in progress? */ 7628 public boolean isRollingUpgrade() { 7629 return rollingUpgradeInfo != null && !rollingUpgradeInfo.isFinalized(); 7630 } 7631 7632 void checkRollingUpgrade(String action) throws RollingUpgradeException { 7633 if (isRollingUpgrade()) { 7634 throw new RollingUpgradeException("Failed to " + action 7635 + " since a rolling upgrade is already in progress." 7636 + " Existing rolling upgrade info:\n" + rollingUpgradeInfo); 7637 } 7638 } 7639 7640 RollingUpgradeInfo finalizeRollingUpgrade() throws IOException { 7641 checkSuperuserPrivilege(); 7642 checkOperation(OperationCategory.WRITE); 7643 writeLock(); 7644 try { 7645 checkOperation(OperationCategory.WRITE); 7646 if (!isRollingUpgrade()) { 7647 return null; 7648 } 7649 checkNameNodeSafeMode("Failed to finalize rolling upgrade"); 7650 7651 finalizeRollingUpgradeInternal(now()); 7652 getEditLog().logFinalizeRollingUpgrade(rollingUpgradeInfo.getFinalizeTime()); 7653 if (haEnabled) { 7654 // roll the edit log to make sure the standby NameNode can tail 7655 getFSImage().rollEditLog(); 7656 } 7657 getFSImage().updateStorageVersion(); 7658 getFSImage().renameCheckpoint(NameNodeFile.IMAGE_ROLLBACK, 7659 NameNodeFile.IMAGE); 7660 } finally { 7661 writeUnlock(); 7662 } 7663 7664 if (!haEnabled) { 7665 // Sync not needed for ha since the edit was rolled after logging. 7666 getEditLog().logSync(); 7667 } 7668 7669 if (auditLog.isInfoEnabled() && isExternalInvocation()) { 7670 logAuditEvent(true, "finalizeRollingUpgrade", null, null, null); 7671 } 7672 return rollingUpgradeInfo; 7673 } 7674 7675 void finalizeRollingUpgradeInternal(long finalizeTime) { 7676 // Set the finalize time 7677 rollingUpgradeInfo.finalize(finalizeTime); 7678 } 7679 7680 long addCacheDirective(CacheDirectiveInfo directive, 7681 EnumSet<CacheFlag> flags, boolean logRetryCache) 7682 throws IOException { 7683 CacheDirectiveInfo effectiveDirective = null; 7684 if (!flags.contains(CacheFlag.FORCE)) { 7685 cacheManager.waitForRescanIfNeeded(); 7686 } 7687 writeLock(); 7688 try { 7689 checkOperation(OperationCategory.WRITE); 7690 if (isInSafeMode()) { 7691 throw new SafeModeException( 7692 "Cannot add cache directive", safeMode); 7693 } 7694 effectiveDirective = FSNDNCacheOp.addCacheDirective(this, cacheManager, 7695 directive, flags, logRetryCache); 7696 } finally { 7697 writeUnlock(); 7698 boolean success = effectiveDirective != null; 7699 if (success) { 7700 getEditLog().logSync(); 7701 } 7702 7703 String effectiveDirectiveStr = effectiveDirective != null ? 7704 effectiveDirective.toString() : null; 7705 logAuditEvent(success, "addCacheDirective", effectiveDirectiveStr, 7706 null, null); 7707 } 7708 return effectiveDirective != null ? effectiveDirective.getId() : 0; 7709 } 7710 7711 void modifyCacheDirective(CacheDirectiveInfo directive, 7712 EnumSet<CacheFlag> flags, boolean logRetryCache) throws IOException { 7713 boolean success = false; 7714 if (!flags.contains(CacheFlag.FORCE)) { 7715 cacheManager.waitForRescanIfNeeded(); 7716 } 7717 writeLock(); 7718 try { 7719 checkOperation(OperationCategory.WRITE); 7720 if (isInSafeMode()) { 7721 throw new SafeModeException( 7722 "Cannot add cache directive", safeMode); 7723 } 7724 FSNDNCacheOp.modifyCacheDirective(this, cacheManager, directive, flags, 7725 logRetryCache); 7726 success = true; 7727 } finally { 7728 writeUnlock(); 7729 if (success) { 7730 getEditLog().logSync(); 7731 } 7732 String idStr = "{id: " + directive.getId().toString() + "}"; 7733 logAuditEvent(success, "modifyCacheDirective", idStr, 7734 directive.toString(), null); 7735 } 7736 } 7737 7738 void removeCacheDirective(long id, boolean logRetryCache) throws IOException { 7739 boolean success = false; 7740 writeLock(); 7741 try { 7742 checkOperation(OperationCategory.WRITE); 7743 if (isInSafeMode()) { 7744 throw new SafeModeException( 7745 "Cannot remove cache directives", safeMode); 7746 } 7747 FSNDNCacheOp.removeCacheDirective(this, cacheManager, id, logRetryCache); 7748 success = true; 7749 } finally { 7750 writeUnlock(); 7751 String idStr = "{id: " + Long.toString(id) + "}"; 7752 logAuditEvent(success, "removeCacheDirective", idStr, null, 7753 null); 7754 } 7755 getEditLog().logSync(); 7756 } 7757 7758 BatchedListEntries<CacheDirectiveEntry> listCacheDirectives( 7759 long startId, CacheDirectiveInfo filter) throws IOException { 7760 checkOperation(OperationCategory.READ); 7761 BatchedListEntries<CacheDirectiveEntry> results; 7762 cacheManager.waitForRescanIfNeeded(); 7763 readLock(); 7764 boolean success = false; 7765 try { 7766 checkOperation(OperationCategory.READ); 7767 results = FSNDNCacheOp.listCacheDirectives(this, cacheManager, startId, 7768 filter); 7769 success = true; 7770 } finally { 7771 readUnlock(); 7772 logAuditEvent(success, "listCacheDirectives", filter.toString(), null, 7773 null); 7774 } 7775 return results; 7776 } 7777 7778 void addCachePool(CachePoolInfo req, boolean logRetryCache) 7779 throws IOException { 7780 writeLock(); 7781 boolean success = false; 7782 String poolInfoStr = null; 7783 try { 7784 checkOperation(OperationCategory.WRITE); 7785 if (isInSafeMode()) { 7786 throw new SafeModeException( 7787 "Cannot add cache pool " + req.getPoolName(), safeMode); 7788 } 7789 CachePoolInfo info = FSNDNCacheOp.addCachePool(this, cacheManager, req, 7790 logRetryCache); 7791 poolInfoStr = info.toString(); 7792 success = true; 7793 } finally { 7794 writeUnlock(); 7795 logAuditEvent(success, "addCachePool", poolInfoStr, null, null); 7796 } 7797 7798 getEditLog().logSync(); 7799 } 7800 7801 void modifyCachePool(CachePoolInfo req, boolean logRetryCache) 7802 throws IOException { 7803 writeLock(); 7804 boolean success = false; 7805 try { 7806 checkOperation(OperationCategory.WRITE); 7807 if (isInSafeMode()) { 7808 throw new SafeModeException( 7809 "Cannot modify cache pool " + req.getPoolName(), safeMode); 7810 } 7811 FSNDNCacheOp.modifyCachePool(this, cacheManager, req, logRetryCache); 7812 success = true; 7813 } finally { 7814 writeUnlock(); 7815 String poolNameStr = "{poolName: " + 7816 (req == null ? null : req.getPoolName()) + "}"; 7817 logAuditEvent(success, "modifyCachePool", poolNameStr, 7818 req == null ? null : req.toString(), null); 7819 } 7820 7821 getEditLog().logSync(); 7822 } 7823 7824 void removeCachePool(String cachePoolName, boolean logRetryCache) 7825 throws IOException { 7826 writeLock(); 7827 boolean success = false; 7828 try { 7829 checkOperation(OperationCategory.WRITE); 7830 if (isInSafeMode()) { 7831 throw new SafeModeException( 7832 "Cannot remove cache pool " + cachePoolName, safeMode); 7833 } 7834 FSNDNCacheOp.removeCachePool(this, cacheManager, cachePoolName, 7835 logRetryCache); 7836 success = true; 7837 } finally { 7838 writeUnlock(); 7839 String poolNameStr = "{poolName: " + cachePoolName + "}"; 7840 logAuditEvent(success, "removeCachePool", poolNameStr, null, null); 7841 } 7842 7843 getEditLog().logSync(); 7844 } 7845 7846 BatchedListEntries<CachePoolEntry> listCachePools(String prevKey) 7847 throws IOException { 7848 BatchedListEntries<CachePoolEntry> results; 7849 checkOperation(OperationCategory.READ); 7850 boolean success = false; 7851 cacheManager.waitForRescanIfNeeded(); 7852 readLock(); 7853 try { 7854 checkOperation(OperationCategory.READ); 7855 results = FSNDNCacheOp.listCachePools(this, cacheManager, prevKey); 7856 success = true; 7857 } finally { 7858 readUnlock(); 7859 logAuditEvent(success, "listCachePools", null, null, null); 7860 } 7861 return results; 7862 } 7863 7864 void modifyAclEntries(final String src, List<AclEntry> aclSpec) 7865 throws IOException { 7866 HdfsFileStatus auditStat = null; 7867 checkOperation(OperationCategory.WRITE); 7868 writeLock(); 7869 try { 7870 checkOperation(OperationCategory.WRITE); 7871 checkNameNodeSafeMode("Cannot modify ACL entries on " + src); 7872 auditStat = FSDirAclOp.modifyAclEntries(dir, src, aclSpec); 7873 } catch (AccessControlException e) { 7874 logAuditEvent(false, "modifyAclEntries", src); 7875 throw e; 7876 } finally { 7877 writeUnlock(); 7878 } 7879 getEditLog().logSync(); 7880 logAuditEvent(true, "modifyAclEntries", src, null, auditStat); 7881 } 7882 7883 void removeAclEntries(final String src, List<AclEntry> aclSpec) 7884 throws IOException { 7885 checkOperation(OperationCategory.WRITE); 7886 HdfsFileStatus auditStat = null; 7887 writeLock(); 7888 try { 7889 checkOperation(OperationCategory.WRITE); 7890 checkNameNodeSafeMode("Cannot remove ACL entries on " + src); 7891 auditStat = FSDirAclOp.removeAclEntries(dir, src, aclSpec); 7892 } catch (AccessControlException e) { 7893 logAuditEvent(false, "removeAclEntries", src); 7894 throw e; 7895 } finally { 7896 writeUnlock(); 7897 } 7898 getEditLog().logSync(); 7899 logAuditEvent(true, "removeAclEntries", src, null, auditStat); 7900 } 7901 7902 void removeDefaultAcl(final String src) throws IOException { 7903 HdfsFileStatus auditStat = null; 7904 checkOperation(OperationCategory.WRITE); 7905 writeLock(); 7906 try { 7907 checkOperation(OperationCategory.WRITE); 7908 checkNameNodeSafeMode("Cannot remove default ACL entries on " + src); 7909 auditStat = FSDirAclOp.removeDefaultAcl(dir, src); 7910 } catch (AccessControlException e) { 7911 logAuditEvent(false, "removeDefaultAcl", src); 7912 throw e; 7913 } finally { 7914 writeUnlock(); 7915 } 7916 getEditLog().logSync(); 7917 logAuditEvent(true, "removeDefaultAcl", src, null, auditStat); 7918 } 7919 7920 void removeAcl(final String src) throws IOException { 7921 HdfsFileStatus auditStat = null; 7922 checkOperation(OperationCategory.WRITE); 7923 writeLock(); 7924 try { 7925 checkOperation(OperationCategory.WRITE); 7926 checkNameNodeSafeMode("Cannot remove ACL on " + src); 7927 auditStat = FSDirAclOp.removeAcl(dir, src); 7928 } catch (AccessControlException e) { 7929 logAuditEvent(false, "removeAcl", src); 7930 throw e; 7931 } finally { 7932 writeUnlock(); 7933 } 7934 getEditLog().logSync(); 7935 logAuditEvent(true, "removeAcl", src, null, auditStat); 7936 } 7937 7938 void setAcl(final String src, List<AclEntry> aclSpec) throws IOException { 7939 HdfsFileStatus auditStat = null; 7940 checkOperation(OperationCategory.WRITE); 7941 writeLock(); 7942 try { 7943 checkOperation(OperationCategory.WRITE); 7944 checkNameNodeSafeMode("Cannot set ACL on " + src); 7945 auditStat = FSDirAclOp.setAcl(dir, src, aclSpec); 7946 } catch (AccessControlException e) { 7947 logAuditEvent(false, "setAcl", src); 7948 throw e; 7949 } finally { 7950 writeUnlock(); 7951 } 7952 getEditLog().logSync(); 7953 logAuditEvent(true, "setAcl", src, null, auditStat); 7954 } 7955 7956 AclStatus getAclStatus(String src) throws IOException { 7957 checkOperation(OperationCategory.READ); 7958 boolean success = false; 7959 readLock(); 7960 try { 7961 checkOperation(OperationCategory.READ); 7962 final AclStatus ret = FSDirAclOp.getAclStatus(dir, src); 7963 success = true; 7964 return ret; 7965 } finally { 7966 readUnlock(); 7967 logAuditEvent(success, "getAclStatus", src); 7968 } 7969 } 7970 7971 /** 7972 * Create an encryption zone on directory src using the specified key. 7973 * 7974 * @param src the path of a directory which will be the root of the 7975 * encryption zone. The directory must be empty. 7976 * @param keyName name of a key which must be present in the configured 7977 * KeyProvider. 7978 * @throws AccessControlException if the caller is not the superuser. 7979 * @throws UnresolvedLinkException if the path can't be resolved. 7980 * @throws SafeModeException if the Namenode is in safe mode. 7981 */ 7982 void createEncryptionZone(final String src, final String keyName, 7983 boolean logRetryCache) 7984 throws IOException, UnresolvedLinkException, 7985 SafeModeException, AccessControlException { 7986 try { 7987 if (provider == null) { 7988 throw new IOException( 7989 "Can't create an encryption zone for " + src + 7990 " since no key provider is available."); 7991 } 7992 if (keyName == null || keyName.isEmpty()) { 7993 throw new IOException("Must specify a key name when creating an " + 7994 "encryption zone"); 7995 } 7996 KeyProvider.Metadata metadata = provider.getMetadata(keyName); 7997 if (metadata == null) { 7998 /* 7999 * It would be nice if we threw something more specific than 8000 * IOException when the key is not found, but the KeyProvider API 8001 * doesn't provide for that. If that API is ever changed to throw 8002 * something more specific (e.g. UnknownKeyException) then we can 8003 * update this to match it, or better yet, just rethrow the 8004 * KeyProvider's exception. 8005 */ 8006 throw new IOException("Key " + keyName + " doesn't exist."); 8007 } 8008 // If the provider supports pool for EDEKs, this will fill in the pool 8009 generateEncryptedDataEncryptionKey(keyName); 8010 createEncryptionZoneInt(src, metadata.getCipher(), 8011 keyName, logRetryCache); 8012 } catch (AccessControlException e) { 8013 logAuditEvent(false, "createEncryptionZone", src); 8014 throw e; 8015 } 8016 } 8017 8018 private void createEncryptionZoneInt(final String srcArg, String cipher, 8019 String keyName, final boolean logRetryCache) throws IOException { 8020 String src = srcArg; 8021 HdfsFileStatus resultingStat = null; 8022 checkSuperuserPrivilege(); 8023 final byte[][] pathComponents = 8024 FSDirectory.getPathComponentsForReservedPath(src); 8025 FSPermissionChecker pc = getPermissionChecker(); 8026 writeLock(); 8027 try { 8028 checkSuperuserPrivilege(); 8029 checkOperation(OperationCategory.WRITE); 8030 checkNameNodeSafeMode("Cannot create encryption zone on " + src); 8031 src = dir.resolvePath(pc, src, pathComponents); 8032 8033 final CipherSuite suite = CipherSuite.convert(cipher); 8034 // For now this is hardcoded, as we only support one method. 8035 final CryptoProtocolVersion version = 8036 CryptoProtocolVersion.ENCRYPTION_ZONES; 8037 final XAttr ezXAttr = dir.createEncryptionZone(src, suite, 8038 version, keyName); 8039 List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1); 8040 xAttrs.add(ezXAttr); 8041 getEditLog().logSetXAttrs(src, xAttrs, logRetryCache); 8042 final INodesInPath iip = dir.getINodesInPath4Write(src, false); 8043 resultingStat = dir.getAuditFileInfo(iip); 8044 } finally { 8045 writeUnlock(); 8046 } 8047 getEditLog().logSync(); 8048 logAuditEvent(true, "createEncryptionZone", srcArg, null, resultingStat); 8049 } 8050 8051 /** 8052 * Get the encryption zone for the specified path. 8053 * 8054 * @param srcArg the path of a file or directory to get the EZ for. 8055 * @return the EZ of the of the path or null if none. 8056 * @throws AccessControlException if the caller is not the superuser. 8057 * @throws UnresolvedLinkException if the path can't be resolved. 8058 */ 8059 EncryptionZone getEZForPath(final String srcArg) 8060 throws AccessControlException, UnresolvedLinkException, IOException { 8061 String src = srcArg; 8062 HdfsFileStatus resultingStat = null; 8063 final byte[][] pathComponents = 8064 FSDirectory.getPathComponentsForReservedPath(src); 8065 boolean success = false; 8066 final FSPermissionChecker pc = getPermissionChecker(); 8067 checkOperation(OperationCategory.READ); 8068 readLock(); 8069 try { 8070 checkOperation(OperationCategory.READ); 8071 src = dir.resolvePath(pc, src, pathComponents); 8072 final INodesInPath iip = dir.getINodesInPath(src, true); 8073 if (isPermissionEnabled) { 8074 dir.checkPathAccess(pc, iip, FsAction.READ); 8075 } 8076 final EncryptionZone ret = dir.getEZForPath(iip); 8077 resultingStat = dir.getAuditFileInfo(iip); 8078 success = true; 8079 return ret; 8080 } finally { 8081 readUnlock(); 8082 logAuditEvent(success, "getEZForPath", srcArg, null, resultingStat); 8083 } 8084 } 8085 8086 BatchedListEntries<EncryptionZone> listEncryptionZones(long prevId) 8087 throws IOException { 8088 boolean success = false; 8089 checkSuperuserPrivilege(); 8090 checkOperation(OperationCategory.READ); 8091 readLock(); 8092 try { 8093 checkSuperuserPrivilege(); 8094 checkOperation(OperationCategory.READ); 8095 final BatchedListEntries<EncryptionZone> ret = 8096 dir.listEncryptionZones(prevId); 8097 success = true; 8098 return ret; 8099 } finally { 8100 readUnlock(); 8101 logAuditEvent(success, "listEncryptionZones", null); 8102 } 8103 } 8104 8105 void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag, 8106 boolean logRetryCache) 8107 throws IOException { 8108 HdfsFileStatus auditStat = null; 8109 writeLock(); 8110 try { 8111 checkOperation(OperationCategory.WRITE); 8112 checkNameNodeSafeMode("Cannot set XAttr on " + src); 8113 auditStat = FSDirXAttrOp.setXAttr(dir, src, xAttr, flag, logRetryCache); 8114 } catch (AccessControlException e) { 8115 logAuditEvent(false, "setXAttr", src); 8116 throw e; 8117 } finally { 8118 writeUnlock(); 8119 } 8120 getEditLog().logSync(); 8121 logAuditEvent(true, "setXAttr", src, null, auditStat); 8122 } 8123 8124 List<XAttr> getXAttrs(final String src, List<XAttr> xAttrs) 8125 throws IOException { 8126 checkOperation(OperationCategory.READ); 8127 readLock(); 8128 try { 8129 checkOperation(OperationCategory.READ); 8130 return FSDirXAttrOp.getXAttrs(dir, src, xAttrs); 8131 } catch (AccessControlException e) { 8132 logAuditEvent(false, "getXAttrs", src); 8133 throw e; 8134 } finally { 8135 readUnlock(); 8136 } 8137 } 8138 8139 List<XAttr> listXAttrs(String src) throws IOException { 8140 checkOperation(OperationCategory.READ); 8141 readLock(); 8142 try { 8143 checkOperation(OperationCategory.READ); 8144 return FSDirXAttrOp.listXAttrs(dir, src); 8145 } catch (AccessControlException e) { 8146 logAuditEvent(false, "listXAttrs", src); 8147 throw e; 8148 } finally { 8149 readUnlock(); 8150 } 8151 } 8152 8153 void removeXAttr(String src, XAttr xAttr, boolean logRetryCache) 8154 throws IOException { 8155 HdfsFileStatus auditStat = null; 8156 writeLock(); 8157 try { 8158 checkOperation(OperationCategory.WRITE); 8159 checkNameNodeSafeMode("Cannot remove XAttr entry on " + src); 8160 auditStat = FSDirXAttrOp.removeXAttr(dir, src, xAttr, logRetryCache); 8161 } catch (AccessControlException e) { 8162 logAuditEvent(false, "removeXAttr", src); 8163 throw e; 8164 } finally { 8165 writeUnlock(); 8166 } 8167 getEditLog().logSync(); 8168 logAuditEvent(true, "removeXAttr", src, null, auditStat); 8169 } 8170 8171 void checkAccess(String src, FsAction mode) throws IOException { 8172 checkOperation(OperationCategory.READ); 8173 byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); 8174 readLock(); 8175 try { 8176 checkOperation(OperationCategory.READ); 8177 src = FSDirectory.resolvePath(src, pathComponents, dir); 8178 final INodesInPath iip = dir.getINodesInPath(src, true); 8179 INode inode = iip.getLastINode(); 8180 if (inode == null) { 8181 throw new FileNotFoundException("Path not found"); 8182 } 8183 if (isPermissionEnabled) { 8184 FSPermissionChecker pc = getPermissionChecker(); 8185 dir.checkPathAccess(pc, iip, mode); 8186 } 8187 } catch (AccessControlException e) { 8188 logAuditEvent(false, "checkAccess", src); 8189 throw e; 8190 } finally { 8191 readUnlock(); 8192 } 8193 } 8194 8195 /** 8196 * Default AuditLogger implementation; used when no access logger is 8197 * defined in the config file. It can also be explicitly listed in the 8198 * config file. 8199 */ 8200 private static class DefaultAuditLogger extends HdfsAuditLogger { 8201 8202 private boolean logTokenTrackingId; 8203 8204 @Override 8205 public void initialize(Configuration conf) { 8206 logTokenTrackingId = conf.getBoolean( 8207 DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_KEY, 8208 DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_DEFAULT); 8209 } 8210 8211 @Override 8212 public void logAuditEvent(boolean succeeded, String userName, 8213 InetAddress addr, String cmd, String src, String dst, 8214 FileStatus status, UserGroupInformation ugi, 8215 DelegationTokenSecretManager dtSecretManager) { 8216 if (auditLog.isInfoEnabled()) { 8217 final StringBuilder sb = auditBuffer.get(); 8218 sb.setLength(0); 8219 sb.append("allowed=").append(succeeded).append("\t"); 8220 sb.append("ugi=").append(userName).append("\t"); 8221 sb.append("ip=").append(addr).append("\t"); 8222 sb.append("cmd=").append(cmd).append("\t"); 8223 sb.append("src=").append(src).append("\t"); 8224 sb.append("dst=").append(dst).append("\t"); 8225 if (null == status) { 8226 sb.append("perm=null"); 8227 } else { 8228 sb.append("perm="); 8229 sb.append(status.getOwner()).append(":"); 8230 sb.append(status.getGroup()).append(":"); 8231 sb.append(status.getPermission()); 8232 } 8233 if (logTokenTrackingId) { 8234 sb.append("\t").append("trackingId="); 8235 String trackingId = null; 8236 if (ugi != null && dtSecretManager != null 8237 && ugi.getAuthenticationMethod() == AuthenticationMethod.TOKEN) { 8238 for (TokenIdentifier tid: ugi.getTokenIdentifiers()) { 8239 if (tid instanceof DelegationTokenIdentifier) { 8240 DelegationTokenIdentifier dtid = 8241 (DelegationTokenIdentifier)tid; 8242 trackingId = dtSecretManager.getTokenTrackingId(dtid); 8243 break; 8244 } 8245 } 8246 } 8247 sb.append(trackingId); 8248 } 8249 sb.append("\t").append("proto="); 8250 sb.append(NamenodeWebHdfsMethods.isWebHdfsInvocation() ? "webhdfs" : "rpc"); 8251 logAuditMessage(sb.toString()); 8252 } 8253 } 8254 8255 public void logAuditMessage(String message) { 8256 auditLog.info(message); 8257 } 8258 } 8259 8260 private static void enableAsyncAuditLog() { 8261 if (!(auditLog instanceof Log4JLogger)) { 8262 LOG.warn("Log4j is required to enable async auditlog"); 8263 return; 8264 } 8265 Logger logger = ((Log4JLogger)auditLog).getLogger(); 8266 @SuppressWarnings("unchecked") 8267 List<Appender> appenders = Collections.list(logger.getAllAppenders()); 8268 // failsafe against trying to async it more than once 8269 if (!appenders.isEmpty() && !(appenders.get(0) instanceof AsyncAppender)) { 8270 AsyncAppender asyncAppender = new AsyncAppender(); 8271 // change logger to have an async appender containing all the 8272 // previously configured appenders 8273 for (Appender appender : appenders) { 8274 logger.removeAppender(appender); 8275 asyncAppender.addAppender(appender); 8276 } 8277 logger.addAppender(asyncAppender); 8278 } 8279 } 8280 8281} 8282