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