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