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