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      final INodeFile pendingFile = fileState.inode;
3074      // Check if the penultimate block is minimally replicated
3075      if (!checkFileProgress(src, pendingFile, false)) {
3076        throw new NotReplicatedYetException("Not replicated yet: " + src);
3077      }
3078      src = fileState.path;
3079
3080      if (onRetryBlock[0] != null && onRetryBlock[0].getLocations().length > 0) {
3081        // This is a retry. No need to generate new locations.
3082        // Use the last block if it has locations.
3083        return null;
3084      }
3085      if (pendingFile.getBlocks().length >= maxBlocksPerFile) {
3086        throw new IOException("File has reached the limit on maximum number of"
3087            + " blocks (" + DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY
3088            + "): " + pendingFile.getBlocks().length + " >= "
3089            + maxBlocksPerFile);
3090      }
3091      blockSize = pendingFile.getPreferredBlockSize();
3092      clientMachine = pendingFile.getFileUnderConstructionFeature()
3093          .getClientMachine();
3094      clientNode = blockManager.getDatanodeManager().getDatanodeByHost(
3095          clientMachine);
3096      replication = pendingFile.getFileReplication();
3097      storagePolicyID = pendingFile.getStoragePolicyID();
3098    } finally {
3099      readUnlock();
3100    }
3101
3102    if (clientNode == null) {
3103      clientNode = getClientNode(clientMachine);
3104    }
3105
3106    // choose targets for the new block to be allocated.
3107    return getBlockManager().chooseTarget4NewBlock( 
3108        src, replication, clientNode, excludedNodes, blockSize, favoredNodes,
3109        storagePolicyID);
3110  }
3111
3112  /**
3113   * Part II of getAdditionalBlock().
3114   * Should repeat the same analysis of the file state as in Part 1,
3115   * but under the write lock.
3116   * If the conditions still hold, then allocate a new block with
3117   * the new targets, add it to the INode and to the BlocksMap.
3118   */
3119  LocatedBlock storeAllocatedBlock(String src, long fileId, String clientName,
3120      ExtendedBlock previous, DatanodeStorageInfo[] targets) throws IOException {
3121    Block newBlock = null;
3122    long offset;
3123    checkOperation(OperationCategory.WRITE);
3124    waitForLoadingFSImage();
3125    writeLock();
3126    try {
3127      checkOperation(OperationCategory.WRITE);
3128      // Run the full analysis again, since things could have changed
3129      // while chooseTarget() was executing.
3130      LocatedBlock[] onRetryBlock = new LocatedBlock[1];
3131      FileState fileState = 
3132          analyzeFileState(src, fileId, clientName, previous, onRetryBlock);
3133      final INodeFile pendingFile = fileState.inode;
3134      src = fileState.path;
3135
3136      if (onRetryBlock[0] != null) {
3137        if (onRetryBlock[0].getLocations().length > 0) {
3138          // This is a retry. Just return the last block if having locations.
3139          return onRetryBlock[0];
3140        } else {
3141          // add new chosen targets to already allocated block and return
3142          BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
3143          ((BlockInfoContiguousUnderConstruction) lastBlockInFile)
3144              .setExpectedLocations(targets);
3145          offset = pendingFile.computeFileSize();
3146          return makeLocatedBlock(lastBlockInFile, targets, offset);
3147        }
3148      }
3149
3150      // commit the last block and complete it if it has minimum replicas
3151      commitOrCompleteLastBlock(pendingFile, fileState.iip,
3152                                ExtendedBlock.getLocalBlock(previous));
3153
3154      // allocate new block, record block locations in INode.
3155      newBlock = createNewBlock();
3156      INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
3157      saveAllocatedBlock(src, inodesInPath, newBlock, targets);
3158
3159      persistNewBlock(src, pendingFile);
3160      offset = pendingFile.computeFileSize();
3161    } finally {
3162      writeUnlock();
3163    }
3164    getEditLog().logSync();
3165
3166    // Return located block
3167    return makeLocatedBlock(newBlock, targets, offset);
3168  }
3169
3170  /*
3171   * Resolve clientmachine address to get a network location path
3172   */
3173  private Node getClientNode(String clientMachine) {
3174    List<String> hosts = new ArrayList<String>(1);
3175    hosts.add(clientMachine);
3176    List<String> rName = getBlockManager().getDatanodeManager()
3177        .resolveNetworkLocation(hosts);
3178    Node clientNode = null;
3179    if (rName != null) {
3180      // Able to resolve clientMachine mapping.
3181      // Create a temp node to findout the rack local nodes
3182      clientNode = new NodeBase(rName.get(0) + NodeBase.PATH_SEPARATOR_STR
3183          + clientMachine);
3184    }
3185    return clientNode;
3186  }
3187
3188  static class FileState {
3189    public final INodeFile inode;
3190    public final String path;
3191    public final INodesInPath iip;
3192
3193    public FileState(INodeFile inode, String fullPath, INodesInPath iip) {
3194      this.inode = inode;
3195      this.path = fullPath;
3196      this.iip = iip;
3197    }
3198  }
3199
3200  FileState analyzeFileState(String src,
3201                                long fileId,
3202                                String clientName,
3203                                ExtendedBlock previous,
3204                                LocatedBlock[] onRetryBlock)
3205          throws IOException  {
3206    assert hasReadLock();
3207
3208    checkBlock(previous);
3209    onRetryBlock[0] = null;
3210    checkNameNodeSafeMode("Cannot add block to " + src);
3211
3212    // have we exceeded the configured limit of fs objects.
3213    checkFsObjectLimit();
3214
3215    Block previousBlock = ExtendedBlock.getLocalBlock(previous);
3216    final INode inode;
3217    final INodesInPath iip;
3218    if (fileId == INodeId.GRANDFATHER_INODE_ID) {
3219      // Older clients may not have given us an inode ID to work with.
3220      // In this case, we have to try to resolve the path and hope it
3221      // hasn't changed or been deleted since the file was opened for write.
3222      iip = dir.getINodesInPath4Write(src);
3223      inode = iip.getLastINode();
3224    } else {
3225      // Newer clients pass the inode ID, so we can just get the inode
3226      // directly.
3227      inode = dir.getInode(fileId);
3228      iip = INodesInPath.fromINode(inode);
3229      if (inode != null) {
3230        src = iip.getPath();
3231      }
3232    }
3233    final INodeFile pendingFile = checkLease(src, clientName, inode, fileId);
3234    BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
3235    if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
3236      // The block that the client claims is the current last block
3237      // doesn't match up with what we think is the last block. There are
3238      // four possibilities:
3239      // 1) This is the first block allocation of an append() pipeline
3240      //    which started appending exactly at or exceeding the block boundary.
3241      //    In this case, the client isn't passed the previous block,
3242      //    so it makes the allocateBlock() call with previous=null.
3243      //    We can distinguish this since the last block of the file
3244      //    will be exactly a full block.
3245      // 2) This is a retry from a client that missed the response of a
3246      //    prior getAdditionalBlock() call, perhaps because of a network
3247      //    timeout, or because of an HA failover. In that case, we know
3248      //    by the fact that the client is re-issuing the RPC that it
3249      //    never began to write to the old block. Hence it is safe to
3250      //    to return the existing block.
3251      // 3) This is an entirely bogus request/bug -- we should error out
3252      //    rather than potentially appending a new block with an empty
3253      //    one in the middle, etc
3254      // 4) This is a retry from a client that timed out while
3255      //    the prior getAdditionalBlock() is still being processed,
3256      //    currently working on chooseTarget(). 
3257      //    There are no means to distinguish between the first and 
3258      //    the second attempts in Part I, because the first one hasn't
3259      //    changed the namesystem state yet.
3260      //    We run this analysis again in Part II where case 4 is impossible.
3261
3262      BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock();
3263      if (previous == null &&
3264          lastBlockInFile != null &&
3265          lastBlockInFile.getNumBytes() >= pendingFile.getPreferredBlockSize() &&
3266          lastBlockInFile.isComplete()) {
3267        // Case 1
3268        NameNode.stateChangeLog.debug(
3269            "BLOCK* NameSystem.allocateBlock: handling block allocation" +
3270            " writing to a file with a complete previous block: src={}" +
3271            " lastBlock={}", src, lastBlockInFile);
3272      } else if (Block.matchingIdAndGenStamp(penultimateBlock, previousBlock)) {
3273        if (lastBlockInFile.getNumBytes() != 0) {
3274          throw new IOException(
3275              "Request looked like a retry to allocate block " +
3276              lastBlockInFile + " but it already contains " +
3277              lastBlockInFile.getNumBytes() + " bytes");
3278        }
3279
3280        // Case 2
3281        // Return the last block.
3282        NameNode.stateChangeLog.info("BLOCK* allocateBlock: " +
3283            "caught retry for allocation of a new block in " +
3284            src + ". Returning previously allocated block " + lastBlockInFile);
3285        long offset = pendingFile.computeFileSize();
3286        onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
3287            ((BlockInfoContiguousUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
3288            offset);
3289        return new FileState(pendingFile, src, iip);
3290      } else {
3291        // Case 3
3292        throw new IOException("Cannot allocate block in " + src + ": " +
3293            "passed 'previous' block " + previous + " does not match actual " +
3294            "last block in file " + lastBlockInFile);
3295      }
3296    }
3297    return new FileState(pendingFile, src, iip);
3298  }
3299
3300  LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs,
3301                                        long offset) throws IOException {
3302    LocatedBlock lBlk = new LocatedBlock(
3303        getExtendedBlock(blk), locs, offset, false);
3304    getBlockManager().setBlockToken(
3305        lBlk, BlockTokenSecretManager.AccessMode.WRITE);
3306    return lBlk;
3307  }
3308
3309  /** @see ClientProtocol#getAdditionalDatanode */
3310  LocatedBlock getAdditionalDatanode(String src, long fileId,
3311      final ExtendedBlock blk, final DatanodeInfo[] existings,
3312      final String[] storageIDs,
3313      final Set<Node> excludes,
3314      final int numAdditionalNodes, final String clientName
3315      ) throws IOException {
3316    //check if the feature is enabled
3317    dtpReplaceDatanodeOnFailure.checkEnabled();
3318
3319    Node clientnode = null;
3320    String clientMachine;
3321    final long preferredblocksize;
3322    final byte storagePolicyID;
3323    final List<DatanodeStorageInfo> chosen;
3324    checkOperation(OperationCategory.READ);
3325    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
3326    FSPermissionChecker pc = getPermissionChecker();
3327    readLock();
3328    try {
3329      checkOperation(OperationCategory.READ);
3330      //check safe mode
3331      checkNameNodeSafeMode("Cannot add datanode; src=" + src + ", blk=" + blk);
3332      src = dir.resolvePath(pc, src, pathComponents);
3333
3334      //check lease
3335      final INode inode;
3336      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
3337        // Older clients may not have given us an inode ID to work with.
3338        // In this case, we have to try to resolve the path and hope it
3339        // hasn't changed or been deleted since the file was opened for write.
3340        inode = dir.getINode(src);
3341      } else {
3342        inode = dir.getInode(fileId);
3343        if (inode != null) src = inode.getFullPathName();
3344      }
3345      final INodeFile file = checkLease(src, clientName, inode, fileId);
3346      clientMachine = file.getFileUnderConstructionFeature().getClientMachine();
3347      clientnode = blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
3348      preferredblocksize = file.getPreferredBlockSize();
3349      storagePolicyID = file.getStoragePolicyID();
3350
3351      //find datanode storages
3352      final DatanodeManager dm = blockManager.getDatanodeManager();
3353      chosen = Arrays.asList(dm.getDatanodeStorageInfos(existings, storageIDs));
3354    } finally {
3355      readUnlock();
3356    }
3357
3358    if (clientnode == null) {
3359      clientnode = getClientNode(clientMachine);
3360    }
3361
3362    // choose new datanodes.
3363    final DatanodeStorageInfo[] targets = blockManager.chooseTarget4AdditionalDatanode(
3364        src, numAdditionalNodes, clientnode, chosen, 
3365        excludes, preferredblocksize, storagePolicyID);
3366    final LocatedBlock lb = new LocatedBlock(blk, targets);
3367    blockManager.setBlockToken(lb, AccessMode.COPY);
3368    return lb;
3369  }
3370
3371  /**
3372   * The client would like to let go of the given block
3373   */
3374  boolean abandonBlock(ExtendedBlock b, long fileId, String src, String holder)
3375      throws IOException {
3376    NameNode.stateChangeLog.debug(
3377        "BLOCK* NameSystem.abandonBlock: {} of file {}", b, src);
3378    checkOperation(OperationCategory.WRITE);
3379    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
3380    FSPermissionChecker pc = getPermissionChecker();
3381    waitForLoadingFSImage();
3382    writeLock();
3383    try {
3384      checkOperation(OperationCategory.WRITE);
3385      checkNameNodeSafeMode("Cannot abandon block " + b + " for file" + src);
3386      src = dir.resolvePath(pc, src, pathComponents);
3387
3388      final INode inode;
3389      final INodesInPath iip;
3390      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
3391        // Older clients may not have given us an inode ID to work with.
3392        // In this case, we have to try to resolve the path and hope it
3393        // hasn't changed or been deleted since the file was opened for write.
3394        iip = dir.getINodesInPath(src, true);
3395        inode = iip.getLastINode();
3396      } else {
3397        inode = dir.getInode(fileId);
3398        iip = INodesInPath.fromINode(inode);
3399        if (inode != null) {
3400          src = iip.getPath();
3401        }
3402      }
3403      final INodeFile file = checkLease(src, holder, inode, fileId);
3404
3405      // Remove the block from the pending creates list
3406      boolean removed = dir.removeBlock(src, iip, file,
3407          ExtendedBlock.getLocalBlock(b));
3408      if (!removed) {
3409        return true;
3410      }
3411      NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: {} is " +
3412          "removed from pendingCreates", b);
3413      persistBlocks(src, file, false);
3414    } finally {
3415      writeUnlock();
3416    }
3417    getEditLog().logSync();
3418
3419    return true;
3420  }
3421
3422  private INodeFile checkLease(String src, String holder, INode inode,
3423      long fileId) throws LeaseExpiredException, FileNotFoundException {
3424    assert hasReadLock();
3425    final String ident = src + " (inode " + fileId + ")";
3426    if (inode == null) {
3427      Lease lease = leaseManager.getLease(holder);
3428      throw new LeaseExpiredException(
3429          "No lease on " + ident + ": File does not exist. "
3430          + (lease != null ? lease.toString()
3431              : "Holder " + holder + " does not have any open files."));
3432    }
3433    if (!inode.isFile()) {
3434      Lease lease = leaseManager.getLease(holder);
3435      throw new LeaseExpiredException(
3436          "No lease on " + ident + ": INode is not a regular file. "
3437              + (lease != null ? lease.toString()
3438              : "Holder " + holder + " does not have any open files."));
3439    }
3440    final INodeFile file = inode.asFile();
3441    if (!file.isUnderConstruction()) {
3442      Lease lease = leaseManager.getLease(holder);
3443      throw new LeaseExpiredException(
3444          "No lease on " + ident + ": File is not open for writing. "
3445          + (lease != null ? lease.toString()
3446              : "Holder " + holder + " does not have any open files."));
3447    }
3448    // No further modification is allowed on a deleted file.
3449    // A file is considered deleted, if it is not in the inodeMap or is marked
3450    // as deleted in the snapshot feature.
3451    if (isFileDeleted(file)) {
3452      throw new FileNotFoundException(src);
3453    }
3454    String clientName = file.getFileUnderConstructionFeature().getClientName();
3455    if (holder != null && !clientName.equals(holder)) {
3456      throw new LeaseExpiredException("Lease mismatch on " + ident +
3457          " owned by " + clientName + " but is accessed by " + holder);
3458    }
3459    return file;
3460  }
3461 
3462  /**
3463   * Complete in-progress write to the given file.
3464   * @return true if successful, false if the client should continue to retry
3465   *         (e.g if not all blocks have reached minimum replication yet)
3466   * @throws IOException on error (eg lease mismatch, file not open, file deleted)
3467   */
3468  boolean completeFile(final String srcArg, String holder,
3469                       ExtendedBlock last, long fileId)
3470    throws SafeModeException, UnresolvedLinkException, IOException {
3471    String src = srcArg;
3472    NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: {} for {}",
3473        src, holder);
3474    checkBlock(last);
3475    boolean success = false;
3476    checkOperation(OperationCategory.WRITE);
3477    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
3478    FSPermissionChecker pc = getPermissionChecker();
3479    waitForLoadingFSImage();
3480    writeLock();
3481    try {
3482      checkOperation(OperationCategory.WRITE);
3483      checkNameNodeSafeMode("Cannot complete file " + src);
3484      src = dir.resolvePath(pc, src, pathComponents);
3485      success = completeFileInternal(src, holder,
3486        ExtendedBlock.getLocalBlock(last), fileId);
3487    } finally {
3488      writeUnlock();
3489    }
3490    getEditLog().logSync();
3491    if (success) {
3492      NameNode.stateChangeLog.info("DIR* completeFile: " + srcArg
3493          + " is closed by " + holder);
3494    }
3495    return success;
3496  }
3497
3498  private boolean completeFileInternal(String src, String holder, Block last,
3499      long fileId) throws IOException {
3500    assert hasWriteLock();
3501    final INodeFile pendingFile;
3502    final INodesInPath iip;
3503    INode inode = null;
3504    try {
3505      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
3506        // Older clients may not have given us an inode ID to work with.
3507        // In this case, we have to try to resolve the path and hope it
3508        // hasn't changed or been deleted since the file was opened for write.
3509        iip = dir.getINodesInPath(src, true);
3510        inode = iip.getLastINode();
3511      } else {
3512        inode = dir.getInode(fileId);
3513        iip = INodesInPath.fromINode(inode);
3514        if (inode != null) {
3515          src = iip.getPath();
3516        }
3517      }
3518      pendingFile = checkLease(src, holder, inode, fileId);
3519    } catch (LeaseExpiredException lee) {
3520      if (inode != null && inode.isFile() &&
3521          !inode.asFile().isUnderConstruction()) {
3522        // This could be a retry RPC - i.e the client tried to close
3523        // the file, but missed the RPC response. Thus, it is trying
3524        // again to close the file. If the file still exists and
3525        // the client's view of the last block matches the actual
3526        // last block, then we'll treat it as a successful close.
3527        // See HDFS-3031.
3528        final Block realLastBlock = inode.asFile().getLastBlock();
3529        if (Block.matchingIdAndGenStamp(last, realLastBlock)) {
3530          NameNode.stateChangeLog.info("DIR* completeFile: " +
3531              "request from " + holder + " to complete inode " + fileId +
3532              "(" + src + ") which is already closed. But, it appears to be " +
3533              "an RPC retry. Returning success");
3534          return true;
3535        }
3536      }
3537      throw lee;
3538    }
3539    // Check the state of the penultimate block. It should be completed
3540    // before attempting to complete the last one.
3541    if (!checkFileProgress(src, pendingFile, false)) {
3542      return false;
3543    }
3544
3545    // commit the last block and complete it if it has minimum replicas
3546    commitOrCompleteLastBlock(pendingFile, iip, last);
3547
3548    if (!checkFileProgress(src, pendingFile, true)) {
3549      return false;
3550    }
3551
3552    finalizeINodeFileUnderConstruction(src, pendingFile,
3553        Snapshot.CURRENT_STATE_ID);
3554    return true;
3555  }
3556
3557  /**
3558   * Save allocated block at the given pending filename
3559   * 
3560   * @param src path to the file
3561   * @param inodesInPath representing each of the components of src.
3562   *                     The last INode is the INode for {@code src} file.
3563   * @param newBlock newly allocated block to be save
3564   * @param targets target datanodes where replicas of the new block is placed
3565   * @throws QuotaExceededException If addition of block exceeds space quota
3566   */
3567  BlockInfoContiguous saveAllocatedBlock(String src, INodesInPath inodesInPath,
3568      Block newBlock, DatanodeStorageInfo[] targets)
3569          throws IOException {
3570    assert hasWriteLock();
3571    BlockInfoContiguous b = dir.addBlock(src, inodesInPath, newBlock, targets);
3572    NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
3573    DatanodeStorageInfo.incrementBlocksScheduled(targets);
3574    return b;
3575  }
3576
3577  /**
3578   * Create new block with a unique block id and a new generation stamp.
3579   */
3580  Block createNewBlock() throws IOException {
3581    assert hasWriteLock();
3582    Block b = new Block(nextBlockId(), 0, 0);
3583    // Increment the generation stamp for every new block.
3584    b.setGenerationStamp(nextGenerationStamp(false));
3585    return b;
3586  }
3587
3588  /**
3589   * Check that the indicated file's blocks are present and
3590   * replicated.  If not, return false. If checkall is true, then check
3591   * all blocks, otherwise check only penultimate block.
3592   */
3593  boolean checkFileProgress(String src, INodeFile v, boolean checkall) {
3594    if (checkall) {
3595      // check all blocks of the file.
3596      for (BlockInfoContiguous block: v.getBlocks()) {
3597        if (!isCompleteBlock(src, block, blockManager.minReplication)) {
3598          return false;
3599        }
3600      }
3601    } else {
3602      // check the penultimate block of this file
3603      BlockInfoContiguous b = v.getPenultimateBlock();
3604      if (b != null
3605          && !isCompleteBlock(src, b, blockManager.minReplication)) {
3606        return false;
3607      }
3608    }
3609    return true;
3610  }
3611
3612  private static boolean isCompleteBlock(String src, BlockInfoContiguous b, int minRepl) {
3613    if (!b.isComplete()) {
3614      final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)b;
3615      final int numNodes = b.numNodes();
3616      LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = "
3617          + uc.getBlockUCState() + ", replication# = " + numNodes
3618          + (numNodes < minRepl? " < ": " >= ")
3619          + " minimum = " + minRepl + ") in file " + src);
3620      return false;
3621    }
3622    return true;
3623  }
3624
3625  ////////////////////////////////////////////////////////////////
3626  // Here's how to handle block-copy failure during client write:
3627  // -- As usual, the client's write should result in a streaming
3628  // backup write to a k-machine sequence.
3629  // -- If one of the backup machines fails, no worries.  Fail silently.
3630  // -- Before client is allowed to close and finalize file, make sure
3631  // that the blocks are backed up.  Namenode may have to issue specific backup
3632  // commands to make up for earlier datanode failures.  Once all copies
3633  // are made, edit namespace and return to client.
3634  ////////////////////////////////////////////////////////////////
3635
3636  /** 
3637   * Change the indicated filename. 
3638   * @deprecated Use {@link #renameTo(String, String, boolean,
3639   * Options.Rename...)} instead.
3640   */
3641  @Deprecated
3642  boolean renameTo(String src, String dst, boolean logRetryCache)
3643      throws IOException {
3644    waitForLoadingFSImage();
3645    FSDirRenameOp.RenameOldResult ret = null;
3646    writeLock();
3647    try {
3648      checkOperation(OperationCategory.WRITE);
3649      checkNameNodeSafeMode("Cannot rename " + src);
3650      ret = FSDirRenameOp.renameToInt(dir, src, dst, logRetryCache);
3651    } catch (AccessControlException e)  {
3652      logAuditEvent(false, "rename", src, dst, null);
3653      throw e;
3654    } finally {
3655      writeUnlock();
3656    }
3657    boolean success = ret != null && ret.success;
3658    if (success) {
3659      getEditLog().logSync();
3660    }
3661    logAuditEvent(success, "rename", src, dst,
3662        ret == null ? null : ret.auditStat);
3663    return success;
3664  }
3665
3666  void renameTo(final String src, final String dst,
3667                boolean logRetryCache, Options.Rename... options)
3668      throws IOException {
3669    waitForLoadingFSImage();
3670    Map.Entry<BlocksMapUpdateInfo, HdfsFileStatus> res = null;
3671    writeLock();
3672    try {
3673      checkOperation(OperationCategory.WRITE);
3674      checkNameNodeSafeMode("Cannot rename " + src);
3675      res = FSDirRenameOp.renameToInt(dir, src, dst, logRetryCache, options);
3676    } catch (AccessControlException e) {
3677      logAuditEvent(false, "rename (options=" + Arrays.toString(options) +
3678          ")", src, dst, null);
3679      throw e;
3680    } finally {
3681      writeUnlock();
3682    }
3683
3684    getEditLog().logSync();
3685
3686    BlocksMapUpdateInfo collectedBlocks = res.getKey();
3687    HdfsFileStatus auditStat = res.getValue();
3688    if (!collectedBlocks.getToDeleteList().isEmpty()) {
3689      removeBlocks(collectedBlocks);
3690      collectedBlocks.clear();
3691    }
3692
3693    logAuditEvent(true, "rename (options=" + Arrays.toString(options) +
3694        ")", src, dst, auditStat);
3695  }
3696
3697  /**
3698   * Remove the indicated file from namespace.
3699   * 
3700   * @see ClientProtocol#delete(String, boolean) for detailed description and 
3701   * description of exceptions
3702   */
3703  boolean delete(String src, boolean recursive, boolean logRetryCache)
3704      throws IOException {
3705    waitForLoadingFSImage();
3706    BlocksMapUpdateInfo toRemovedBlocks = null;
3707    writeLock();
3708    boolean ret = false;
3709    try {
3710      checkOperation(OperationCategory.WRITE);
3711      checkNameNodeSafeMode("Cannot delete " + src);
3712      toRemovedBlocks = FSDirDeleteOp.delete(
3713          this, src, recursive, logRetryCache);
3714      ret = toRemovedBlocks != null;
3715    } catch (AccessControlException e) {
3716      logAuditEvent(false, "delete", src);
3717      throw e;
3718    } finally {
3719      writeUnlock();
3720    }
3721    getEditLog().logSync();
3722    if (toRemovedBlocks != null) {
3723      removeBlocks(toRemovedBlocks); // Incremental deletion of blocks
3724    }
3725    logAuditEvent(true, "delete", src);
3726    return ret;
3727  }
3728
3729  FSPermissionChecker getPermissionChecker()
3730      throws AccessControlException {
3731    return dir.getPermissionChecker();
3732  }
3733
3734  /**
3735   * From the given list, incrementally remove the blocks from blockManager
3736   * Writelock is dropped and reacquired every BLOCK_DELETION_INCREMENT to
3737   * ensure that other waiters on the lock can get in. See HDFS-2938
3738   * 
3739   * @param blocks
3740   *          An instance of {@link BlocksMapUpdateInfo} which contains a list
3741   *          of blocks that need to be removed from blocksMap
3742   */
3743  void removeBlocks(BlocksMapUpdateInfo blocks) {
3744    List<Block> toDeleteList = blocks.getToDeleteList();
3745    Iterator<Block> iter = toDeleteList.iterator();
3746    while (iter.hasNext()) {
3747      writeLock();
3748      try {
3749        for (int i = 0; i < BLOCK_DELETION_INCREMENT && iter.hasNext(); i++) {
3750          blockManager.removeBlock(iter.next());
3751        }
3752      } finally {
3753        writeUnlock();
3754      }
3755    }
3756  }
3757  
3758  /**
3759   * Remove leases and inodes related to a given path
3760   * @param src The given path
3761   * @param removedINodes Containing the list of inodes to be removed from
3762   *                      inodesMap
3763   * @param acquireINodeMapLock Whether to acquire the lock for inode removal
3764   */
3765  void removeLeasesAndINodes(String src, List<INode> removedINodes,
3766      final boolean acquireINodeMapLock) {
3767    assert hasWriteLock();
3768    leaseManager.removeLeaseWithPrefixPath(src);
3769    // remove inodes from inodesMap
3770    if (removedINodes != null) {
3771      if (acquireINodeMapLock) {
3772        dir.writeLock();
3773      }
3774      try {
3775        dir.removeFromInodeMap(removedINodes);
3776      } finally {
3777        if (acquireINodeMapLock) {
3778          dir.writeUnlock();
3779        }
3780      }
3781      removedINodes.clear();
3782    }
3783  }
3784
3785  /**
3786   * Removes the blocks from blocksmap and updates the safemode blocks total
3787   * 
3788   * @param blocks
3789   *          An instance of {@link BlocksMapUpdateInfo} which contains a list
3790   *          of blocks that need to be removed from blocksMap
3791   */
3792  void removeBlocksAndUpdateSafemodeTotal(BlocksMapUpdateInfo blocks) {
3793    assert hasWriteLock();
3794    // In the case that we are a Standby tailing edits from the
3795    // active while in safe-mode, we need to track the total number
3796    // of blocks and safe blocks in the system.
3797    boolean trackBlockCounts = isSafeModeTrackingBlocks();
3798    int numRemovedComplete = 0, numRemovedSafe = 0;
3799
3800    for (Block b : blocks.getToDeleteList()) {
3801      if (trackBlockCounts) {
3802        BlockInfoContiguous bi = getStoredBlock(b);
3803        if (bi.isComplete()) {
3804          numRemovedComplete++;
3805          if (bi.numNodes() >= blockManager.minReplication) {
3806            numRemovedSafe++;
3807          }
3808        }
3809      }
3810      blockManager.removeBlock(b);
3811    }
3812    if (trackBlockCounts) {
3813      if (LOG.isDebugEnabled()) {
3814        LOG.debug("Adjusting safe-mode totals for deletion."
3815            + "decreasing safeBlocks by " + numRemovedSafe
3816            + ", totalBlocks by " + numRemovedComplete);
3817      }
3818      adjustSafeModeBlockTotals(-numRemovedSafe, -numRemovedComplete);
3819    }
3820  }
3821
3822  /**
3823   * @see SafeModeInfo#shouldIncrementallyTrackBlocks
3824   */
3825  private boolean isSafeModeTrackingBlocks() {
3826    if (!haEnabled) {
3827      // Never track blocks incrementally in non-HA code.
3828      return false;
3829    }
3830    SafeModeInfo sm = this.safeMode;
3831    return sm != null && sm.shouldIncrementallyTrackBlocks();
3832  }
3833
3834  /**
3835   * Get the file info for a specific file.
3836   *
3837   * @param src The string representation of the path to the file
3838   * @param resolveLink whether to throw UnresolvedLinkException
3839   *        if src refers to a symlink
3840   *
3841   * @throws AccessControlException if access is denied
3842   * @throws UnresolvedLinkException if a symlink is encountered.
3843   *
3844   * @return object containing information regarding the file
3845   *         or null if file not found
3846   * @throws StandbyException
3847   */
3848  HdfsFileStatus getFileInfo(final String src, boolean resolveLink)
3849    throws IOException {
3850    checkOperation(OperationCategory.READ);
3851    HdfsFileStatus stat = null;
3852    readLock();
3853    try {
3854      checkOperation(OperationCategory.READ);
3855      stat = FSDirStatAndListingOp.getFileInfo(dir, src, resolveLink);
3856    } catch (AccessControlException e) {
3857      logAuditEvent(false, "getfileinfo", src);
3858      throw e;
3859    } finally {
3860      readUnlock();
3861    }
3862    logAuditEvent(true, "getfileinfo", src);
3863    return stat;
3864  }
3865
3866  /**
3867   * Returns true if the file is closed
3868   */
3869  boolean isFileClosed(final String src) throws IOException {
3870    checkOperation(OperationCategory.READ);
3871    readLock();
3872    try {
3873      checkOperation(OperationCategory.READ);
3874      return FSDirStatAndListingOp.isFileClosed(dir, src);
3875    } catch (AccessControlException e) {
3876      logAuditEvent(false, "isFileClosed", src);
3877      throw e;
3878    } finally {
3879      readUnlock();
3880    }
3881  }
3882
3883  /**
3884   * Create all the necessary directories
3885   */
3886  boolean mkdirs(String src, PermissionStatus permissions,
3887      boolean createParent) throws IOException {
3888    HdfsFileStatus auditStat = null;
3889    checkOperation(OperationCategory.WRITE);
3890    writeLock();
3891    try {
3892      checkOperation(OperationCategory.WRITE);
3893      checkNameNodeSafeMode("Cannot create directory " + src);
3894      auditStat = FSDirMkdirOp.mkdirs(this, src, permissions, createParent);
3895    } catch (AccessControlException e) {
3896      logAuditEvent(false, "mkdirs", src);
3897      throw e;
3898    } finally {
3899      writeUnlock();
3900    }
3901    getEditLog().logSync();
3902    logAuditEvent(true, "mkdirs", src, null, auditStat);
3903    return true;
3904  }
3905
3906  /**
3907   * Get the content summary for a specific file/dir.
3908   *
3909   * @param src The string representation of the path to the file
3910   *
3911   * @throws AccessControlException if access is denied
3912   * @throws UnresolvedLinkException if a symlink is encountered.
3913   * @throws FileNotFoundException if no file exists
3914   * @throws StandbyException
3915   * @throws IOException for issues with writing to the audit log
3916   *
3917   * @return object containing information regarding the file
3918   *         or null if file not found
3919   */
3920  ContentSummary getContentSummary(final String src) throws IOException {
3921    checkOperation(OperationCategory.READ);
3922    readLock();
3923    boolean success = true;
3924    try {
3925      checkOperation(OperationCategory.READ);
3926      return FSDirStatAndListingOp.getContentSummary(dir, src);
3927    } catch (AccessControlException ace) {
3928      success = false;
3929      throw ace;
3930    } finally {
3931      readUnlock();
3932      logAuditEvent(success, "contentSummary", src);
3933    }
3934  }
3935
3936  /**
3937   * Set the namespace quota and storage space quota for a directory.
3938   * See {@link ClientProtocol#setQuota(String, long, long, StorageType)} for the
3939   * contract.
3940   * 
3941   * Note: This does not support ".inodes" relative path.
3942   */
3943  void setQuota(String src, long nsQuota, long ssQuota, StorageType type)
3944      throws IOException {
3945    checkOperation(OperationCategory.WRITE);
3946    writeLock();
3947    boolean success = false;
3948    try {
3949      checkOperation(OperationCategory.WRITE);
3950      checkNameNodeSafeMode("Cannot set quota on " + src);
3951      FSDirAttrOp.setQuota(dir, src, nsQuota, ssQuota, type);
3952      success = true;
3953    } finally {
3954      writeUnlock();
3955      if (success) {
3956        getEditLog().logSync();
3957      }
3958      logAuditEvent(success, "setQuota", src);
3959    }
3960  }
3961
3962  /** Persist all metadata about this file.
3963   * @param src The string representation of the path
3964   * @param fileId The inode ID that we're fsyncing.  Older clients will pass
3965   *               INodeId.GRANDFATHER_INODE_ID here.
3966   * @param clientName The string representation of the client
3967   * @param lastBlockLength The length of the last block 
3968   *                        under construction reported from client.
3969   * @throws IOException if path does not exist
3970   */
3971  void fsync(String src, long fileId, String clientName, long lastBlockLength)
3972      throws IOException {
3973    NameNode.stateChangeLog.info("BLOCK* fsync: " + src + " for " + clientName);
3974    checkOperation(OperationCategory.WRITE);
3975    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
3976
3977    FSPermissionChecker pc = getPermissionChecker();
3978    waitForLoadingFSImage();
3979    writeLock();
3980    try {
3981      checkOperation(OperationCategory.WRITE);
3982      checkNameNodeSafeMode("Cannot fsync file " + src);
3983      src = dir.resolvePath(pc, src, pathComponents);
3984      final INode inode;
3985      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
3986        // Older clients may not have given us an inode ID to work with.
3987        // In this case, we have to try to resolve the path and hope it
3988        // hasn't changed or been deleted since the file was opened for write.
3989        inode = dir.getINode(src);
3990      } else {
3991        inode = dir.getInode(fileId);
3992        if (inode != null) src = inode.getFullPathName();
3993      }
3994      final INodeFile pendingFile = checkLease(src, clientName, inode, fileId);
3995      if (lastBlockLength > 0) {
3996        pendingFile.getFileUnderConstructionFeature().updateLengthOfLastBlock(
3997            pendingFile, lastBlockLength);
3998      }
3999      persistBlocks(src, pendingFile, false);
4000    } finally {
4001      writeUnlock();
4002    }
4003    getEditLog().logSync();
4004  }
4005
4006  /**
4007   * Move a file that is being written to be immutable.
4008   * @param src The filename
4009   * @param lease The lease for the client creating the file
4010   * @param recoveryLeaseHolder reassign lease to this holder if the last block
4011   *        needs recovery; keep current holder if null.
4012   * @throws AlreadyBeingCreatedException if file is waiting to achieve minimal
4013   *         replication;<br>
4014   *         RecoveryInProgressException if lease recovery is in progress.<br>
4015   *         IOException in case of an error.
4016   * @return true  if file has been successfully finalized and closed or 
4017   *         false if block recovery has been initiated. Since the lease owner
4018   *         has been changed and logged, caller should call logSync().
4019   */
4020  boolean internalReleaseLease(Lease lease, String src, INodesInPath iip,
4021      String recoveryLeaseHolder) throws IOException {
4022    LOG.info("Recovering " + lease + ", src=" + src);
4023    assert !isInSafeMode();
4024    assert hasWriteLock();
4025
4026    final INodeFile pendingFile = iip.getLastINode().asFile();
4027    int nrBlocks = pendingFile.numBlocks();
4028    BlockInfoContiguous[] blocks = pendingFile.getBlocks();
4029
4030    int nrCompleteBlocks;
4031    BlockInfoContiguous curBlock = null;
4032    for(nrCompleteBlocks = 0; nrCompleteBlocks < nrBlocks; nrCompleteBlocks++) {
4033      curBlock = blocks[nrCompleteBlocks];
4034      if(!curBlock.isComplete())
4035        break;
4036      assert blockManager.checkMinReplication(curBlock) :
4037              "A COMPLETE block is not minimally replicated in " + src;
4038    }
4039
4040    // If there are no incomplete blocks associated with this file,
4041    // then reap lease immediately and close the file.
4042    if(nrCompleteBlocks == nrBlocks) {
4043      finalizeINodeFileUnderConstruction(src, pendingFile,
4044          iip.getLatestSnapshotId());
4045      NameNode.stateChangeLog.warn("BLOCK*"
4046        + " internalReleaseLease: All existing blocks are COMPLETE,"
4047        + " lease removed, file closed.");
4048      return true;  // closed!
4049    }
4050
4051    // Only the last and the penultimate blocks may be in non COMPLETE state.
4052    // If the penultimate block is not COMPLETE, then it must be COMMITTED.
4053    if(nrCompleteBlocks < nrBlocks - 2 ||
4054       nrCompleteBlocks == nrBlocks - 2 &&
4055         curBlock != null &&
4056         curBlock.getBlockUCState() != BlockUCState.COMMITTED) {
4057      final String message = "DIR* NameSystem.internalReleaseLease: "
4058        + "attempt to release a create lock on "
4059        + src + " but file is already closed.";
4060      NameNode.stateChangeLog.warn(message);
4061      throw new IOException(message);
4062    }
4063
4064    // The last block is not COMPLETE, and
4065    // that the penultimate block if exists is either COMPLETE or COMMITTED
4066    final BlockInfoContiguous lastBlock = pendingFile.getLastBlock();
4067    BlockUCState lastBlockState = lastBlock.getBlockUCState();
4068    BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock();
4069
4070    // If penultimate block doesn't exist then its minReplication is met
4071    boolean penultimateBlockMinReplication = penultimateBlock == null ? true :
4072        blockManager.checkMinReplication(penultimateBlock);
4073
4074    switch(lastBlockState) {
4075    case COMPLETE:
4076      assert false : "Already checked that the last block is incomplete";
4077      break;
4078    case COMMITTED:
4079      // Close file if committed blocks are minimally replicated
4080      if(penultimateBlockMinReplication &&
4081          blockManager.checkMinReplication(lastBlock)) {
4082        finalizeINodeFileUnderConstruction(src, pendingFile,
4083            iip.getLatestSnapshotId());
4084        NameNode.stateChangeLog.warn("BLOCK*"
4085          + " internalReleaseLease: Committed blocks are minimally replicated,"
4086          + " lease removed, file closed.");
4087        return true;  // closed!
4088      }
4089      // Cannot close file right now, since some blocks 
4090      // are not yet minimally replicated.
4091      // This may potentially cause infinite loop in lease recovery
4092      // if there are no valid replicas on data-nodes.
4093      String message = "DIR* NameSystem.internalReleaseLease: " +
4094          "Failed to release lease for file " + src +
4095          ". Committed blocks are waiting to be minimally replicated." +
4096          " Try again later.";
4097      NameNode.stateChangeLog.warn(message);
4098      throw new AlreadyBeingCreatedException(message);
4099    case UNDER_CONSTRUCTION:
4100    case UNDER_RECOVERY:
4101      final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)lastBlock;
4102      // determine if last block was intended to be truncated
4103      Block recoveryBlock = uc.getTruncateBlock();
4104      boolean truncateRecovery = recoveryBlock != null;
4105      boolean copyOnTruncate = truncateRecovery &&
4106          recoveryBlock.getBlockId() != uc.getBlockId();
4107      assert !copyOnTruncate ||
4108          recoveryBlock.getBlockId() < uc.getBlockId() &&
4109          recoveryBlock.getGenerationStamp() < uc.getGenerationStamp() &&
4110          recoveryBlock.getNumBytes() > uc.getNumBytes() :
4111            "wrong recoveryBlock";
4112
4113      // setup the last block locations from the blockManager if not known
4114      if (uc.getNumExpectedLocations() == 0) {
4115        uc.setExpectedLocations(blockManager.getStorages(lastBlock));
4116      }
4117
4118      if (uc.getNumExpectedLocations() == 0 && uc.getNumBytes() == 0) {
4119        // There is no datanode reported to this block.
4120        // may be client have crashed before writing data to pipeline.
4121        // This blocks doesn't need any recovery.
4122        // We can remove this block and close the file.
4123        pendingFile.removeLastBlock(lastBlock);
4124        finalizeINodeFileUnderConstruction(src, pendingFile,
4125            iip.getLatestSnapshotId());
4126        NameNode.stateChangeLog.warn("BLOCK* internalReleaseLease: "
4127            + "Removed empty last block and closed file.");
4128        return true;
4129      }
4130      // start recovery of the last block for this file
4131      long blockRecoveryId = nextGenerationStamp(blockIdManager.isLegacyBlock(uc));
4132      lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
4133      if(copyOnTruncate) {
4134        uc.setGenerationStamp(blockRecoveryId);
4135      } else if(truncateRecovery) {
4136        recoveryBlock.setGenerationStamp(blockRecoveryId);
4137      }
4138      uc.initializeBlockRecovery(blockRecoveryId);
4139      leaseManager.renewLease(lease);
4140      // Cannot close file right now, since the last block requires recovery.
4141      // This may potentially cause infinite loop in lease recovery
4142      // if there are no valid replicas on data-nodes.
4143      NameNode.stateChangeLog.warn(
4144                "DIR* NameSystem.internalReleaseLease: " +
4145                "File " + src + " has not been closed." +
4146               " Lease recovery is in progress. " +
4147                "RecoveryId = " + blockRecoveryId + " for block " + lastBlock);
4148      break;
4149    }
4150    return false;
4151  }
4152
4153  private Lease reassignLease(Lease lease, String src, String newHolder,
4154      INodeFile pendingFile) {
4155    assert hasWriteLock();
4156    if(newHolder == null)
4157      return lease;
4158    // The following transaction is not synced. Make sure it's sync'ed later.
4159    logReassignLease(lease.getHolder(), src, newHolder);
4160    return reassignLeaseInternal(lease, src, newHolder, pendingFile);
4161  }
4162  
4163  Lease reassignLeaseInternal(Lease lease, String src, String newHolder,
4164      INodeFile pendingFile) {
4165    assert hasWriteLock();
4166    pendingFile.getFileUnderConstructionFeature().setClientName(newHolder);
4167    return leaseManager.reassignLease(lease, src, newHolder);
4168  }
4169
4170  private void commitOrCompleteLastBlock(final INodeFile fileINode,
4171      final INodesInPath iip, final Block commitBlock) throws IOException {
4172    assert hasWriteLock();
4173    Preconditions.checkArgument(fileINode.isUnderConstruction());
4174    if (!blockManager.commitOrCompleteLastBlock(fileINode, commitBlock)) {
4175      return;
4176    }
4177
4178    // Adjust disk space consumption if required
4179    final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();    
4180    if (diff > 0) {
4181      try {
4182        dir.updateSpaceConsumed(iip, 0, -diff, fileINode.getFileReplication());
4183      } catch (IOException e) {
4184        LOG.warn("Unexpected exception while updating disk space.", e);
4185      }
4186    }
4187  }
4188
4189  private void finalizeINodeFileUnderConstruction(String src,
4190      INodeFile pendingFile, int latestSnapshot) throws IOException {
4191    assert hasWriteLock();
4192
4193    FileUnderConstructionFeature uc = pendingFile.getFileUnderConstructionFeature();
4194    if (uc == null) {
4195      throw new IOException("Cannot finalize file " + src
4196          + " because it is not under construction");
4197    }
4198    leaseManager.removeLease(uc.getClientName(), src);
4199    
4200    pendingFile.recordModification(latestSnapshot);
4201
4202    // The file is no longer pending.
4203    // Create permanent INode, update blocks. No need to replace the inode here
4204    // since we just remove the uc feature from pendingFile
4205    pendingFile.toCompleteFile(now());
4206
4207    waitForLoadingFSImage();
4208    // close file and persist block allocations for this file
4209    closeFile(src, pendingFile);
4210
4211    blockManager.checkReplication(pendingFile);
4212  }
4213
4214  @VisibleForTesting
4215  BlockInfoContiguous getStoredBlock(Block block) {
4216    return blockManager.getStoredBlock(block);
4217  }
4218  
4219  @Override
4220  public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC) {
4221    assert hasReadLock();
4222    final BlockCollection bc = blockUC.getBlockCollection();
4223    if (bc == null || !(bc instanceof INodeFile)
4224        || !bc.isUnderConstruction()) {
4225      return false;
4226    }
4227
4228    String fullName = bc.getName();
4229    try {
4230      if (fullName != null && fullName.startsWith(Path.SEPARATOR)
4231          && dir.getINode(fullName) == bc) {
4232        // If file exists in normal path then no need to look in snapshot
4233        return false;
4234      }
4235    } catch (UnresolvedLinkException e) {
4236      LOG.error("Error while resolving the link : " + fullName, e);
4237      return false;
4238    }
4239    /*
4240     * 1. if bc is under construction and also with snapshot, and
4241     * bc is not in the current fsdirectory tree, bc must represent a snapshot
4242     * file. 
4243     * 2. if fullName is not an absolute path, bc cannot be existent in the 
4244     * current fsdirectory tree. 
4245     * 3. if bc is not the current node associated with fullName, bc must be a
4246     * snapshot inode.
4247     */
4248    return true;
4249  }
4250
4251  void commitBlockSynchronization(ExtendedBlock oldBlock,
4252      long newgenerationstamp, long newlength,
4253      boolean closeFile, boolean deleteblock, DatanodeID[] newtargets,
4254      String[] newtargetstorages) throws IOException {
4255    LOG.info("commitBlockSynchronization(oldBlock=" + oldBlock
4256             + ", newgenerationstamp=" + newgenerationstamp
4257             + ", newlength=" + newlength
4258             + ", newtargets=" + Arrays.asList(newtargets)
4259             + ", closeFile=" + closeFile
4260             + ", deleteBlock=" + deleteblock
4261             + ")");
4262    checkOperation(OperationCategory.WRITE);
4263    String src = "";
4264    waitForLoadingFSImage();
4265    writeLock();
4266    try {
4267      checkOperation(OperationCategory.WRITE);
4268      // If a DN tries to commit to the standby, the recovery will
4269      // fail, and the next retry will succeed on the new NN.
4270  
4271      checkNameNodeSafeMode(
4272          "Cannot commitBlockSynchronization while in safe mode");
4273      final BlockInfoContiguous storedBlock = getStoredBlock(
4274          ExtendedBlock.getLocalBlock(oldBlock));
4275      if (storedBlock == null) {
4276        if (deleteblock) {
4277          // This may be a retry attempt so ignore the failure
4278          // to locate the block.
4279          if (LOG.isDebugEnabled()) {
4280            LOG.debug("Block (=" + oldBlock + ") not found");
4281          }
4282          return;
4283        } else {
4284          throw new IOException("Block (=" + oldBlock + ") not found");
4285        }
4286      }
4287      final long oldGenerationStamp = storedBlock.getGenerationStamp();
4288      final long oldNumBytes = storedBlock.getNumBytes();
4289      //
4290      // The implementation of delete operation (see @deleteInternal method)
4291      // first removes the file paths from namespace, and delays the removal
4292      // of blocks to later time for better performance. When
4293      // commitBlockSynchronization (this method) is called in between, the
4294      // blockCollection of storedBlock could have been assigned to null by
4295      // the delete operation, throw IOException here instead of NPE; if the
4296      // file path is already removed from namespace by the delete operation,
4297      // throw FileNotFoundException here, so not to proceed to the end of
4298      // this method to add a CloseOp to the edit log for an already deleted
4299      // file (See HDFS-6825).
4300      //
4301      BlockCollection blockCollection = storedBlock.getBlockCollection();
4302      if (blockCollection == null) {
4303        throw new IOException("The blockCollection of " + storedBlock
4304            + " is null, likely because the file owning this block was"
4305            + " deleted and the block removal is delayed");
4306      }
4307      INodeFile iFile = ((INode)blockCollection).asFile();
4308      if (isFileDeleted(iFile)) {
4309        throw new FileNotFoundException("File not found: "
4310            + iFile.getFullPathName() + ", likely due to delayed block"
4311            + " removal");
4312      }
4313      if ((!iFile.isUnderConstruction() || storedBlock.isComplete()) &&
4314          iFile.getLastBlock().isComplete()) {
4315        if (LOG.isDebugEnabled()) {
4316          LOG.debug("Unexpected block (=" + oldBlock
4317                    + ") since the file (=" + iFile.getLocalName()
4318                    + ") is not under construction");
4319        }
4320        return;
4321      }
4322
4323      BlockInfoContiguousUnderConstruction truncatedBlock =
4324          (BlockInfoContiguousUnderConstruction) iFile.getLastBlock();
4325      long recoveryId = truncatedBlock.getBlockRecoveryId();
4326      boolean copyTruncate =
4327          truncatedBlock.getBlockId() != storedBlock.getBlockId();
4328      if(recoveryId != newgenerationstamp) {
4329        throw new IOException("The recovery id " + newgenerationstamp
4330                              + " does not match current recovery id "
4331                              + recoveryId + " for block " + oldBlock);
4332      }
4333
4334      if (deleteblock) {
4335        Block blockToDel = ExtendedBlock.getLocalBlock(oldBlock);
4336        boolean remove = iFile.removeLastBlock(blockToDel);
4337        if (remove) {
4338          blockManager.removeBlock(storedBlock);
4339        }
4340      }
4341      else {
4342        // update last block
4343        if(!copyTruncate) {
4344          storedBlock.setGenerationStamp(newgenerationstamp);
4345          storedBlock.setNumBytes(newlength);
4346        }
4347
4348        // find the DatanodeDescriptor objects
4349        ArrayList<DatanodeDescriptor> trimmedTargets =
4350            new ArrayList<DatanodeDescriptor>(newtargets.length);
4351        ArrayList<String> trimmedStorages =
4352            new ArrayList<String>(newtargets.length);
4353        if (newtargets.length > 0) {
4354          for (int i = 0; i < newtargets.length; ++i) {
4355            // try to get targetNode
4356            DatanodeDescriptor targetNode =
4357                blockManager.getDatanodeManager().getDatanode(newtargets[i]);
4358            if (targetNode != null) {
4359              trimmedTargets.add(targetNode);
4360              trimmedStorages.add(newtargetstorages[i]);
4361            } else if (LOG.isDebugEnabled()) {
4362              LOG.debug("DatanodeDescriptor (=" + newtargets[i] + ") not found");
4363            }
4364          }
4365        }
4366        if ((closeFile) && !trimmedTargets.isEmpty()) {
4367          // the file is getting closed. Insert block locations into blockManager.
4368          // Otherwise fsck will report these blocks as MISSING, especially if the
4369          // blocksReceived from Datanodes take a long time to arrive.
4370          for (int i = 0; i < trimmedTargets.size(); i++) {
4371            DatanodeStorageInfo storageInfo =
4372                trimmedTargets.get(i).getStorageInfo(trimmedStorages.get(i));
4373            if (storageInfo != null) {
4374              if(copyTruncate) {
4375                storageInfo.addBlock(truncatedBlock);
4376              } else {
4377                storageInfo.addBlock(storedBlock);
4378              }
4379            }
4380          }
4381        }
4382
4383        // add pipeline locations into the INodeUnderConstruction
4384        DatanodeStorageInfo[] trimmedStorageInfos =
4385            blockManager.getDatanodeManager().getDatanodeStorageInfos(
4386                trimmedTargets.toArray(new DatanodeID[trimmedTargets.size()]),
4387                trimmedStorages.toArray(new String[trimmedStorages.size()]));
4388        if(copyTruncate) {
4389          iFile.setLastBlock(truncatedBlock, trimmedStorageInfos);
4390        } else {
4391          iFile.setLastBlock(storedBlock, trimmedStorageInfos);
4392          if (closeFile) {
4393            blockManager.markBlockReplicasAsCorrupt(storedBlock,
4394                oldGenerationStamp, oldNumBytes, trimmedStorageInfos);
4395          }
4396        }
4397      }
4398
4399      if (closeFile) {
4400        if(copyTruncate) {
4401          src = closeFileCommitBlocks(iFile, truncatedBlock);
4402          if(!iFile.isBlockInLatestSnapshot(storedBlock)) {
4403            blockManager.removeBlock(storedBlock);
4404          }
4405        } else {
4406          src = closeFileCommitBlocks(iFile, storedBlock);
4407        }
4408      } else {
4409        // If this commit does not want to close the file, persist blocks
4410        src = iFile.getFullPathName();
4411        persistBlocks(src, iFile, false);
4412      }
4413    } finally {
4414      writeUnlock();
4415    }
4416    getEditLog().logSync();
4417    if (closeFile) {
4418      LOG.info("commitBlockSynchronization(oldBlock=" + oldBlock
4419          + ", file=" + src
4420          + ", newgenerationstamp=" + newgenerationstamp
4421          + ", newlength=" + newlength
4422          + ", newtargets=" + Arrays.asList(newtargets) + ") successful");
4423    } else {
4424      LOG.info("commitBlockSynchronization(" + oldBlock + ") successful");
4425    }
4426  }
4427
4428  /**
4429   * @param pendingFile open file that needs to be closed
4430   * @param storedBlock last block
4431   * @return Path of the file that was closed.
4432   * @throws IOException on error
4433   */
4434  @VisibleForTesting
4435  String closeFileCommitBlocks(INodeFile pendingFile, BlockInfoContiguous storedBlock)
4436      throws IOException {
4437    final INodesInPath iip = INodesInPath.fromINode(pendingFile);
4438    final String src = iip.getPath();
4439
4440    // commit the last block and complete it if it has minimum replicas
4441    commitOrCompleteLastBlock(pendingFile, iip, storedBlock);
4442
4443    //remove lease, close file
4444    finalizeINodeFileUnderConstruction(src, pendingFile,
4445        Snapshot.findLatestSnapshot(pendingFile, Snapshot.CURRENT_STATE_ID));
4446
4447    return src;
4448  }
4449
4450  /**
4451   * Renew the lease(s) held by the given client
4452   */
4453  void renewLease(String holder) throws IOException {
4454    checkOperation(OperationCategory.WRITE);
4455    readLock();
4456    try {
4457      checkOperation(OperationCategory.WRITE);
4458      checkNameNodeSafeMode("Cannot renew lease for " + holder);
4459      leaseManager.renewLease(holder);
4460    } finally {
4461      readUnlock();
4462    }
4463  }
4464
4465  /**
4466   * Get a partial listing of the indicated directory
4467   *
4468   * @param src the directory name
4469   * @param startAfter the name to start after
4470   * @param needLocation if blockLocations need to be returned
4471   * @return a partial listing starting after startAfter
4472   * 
4473   * @throws AccessControlException if access is denied
4474   * @throws UnresolvedLinkException if symbolic link is encountered
4475   * @throws IOException if other I/O error occurred
4476   */
4477  DirectoryListing getListing(String src, byte[] startAfter,
4478      boolean needLocation) 
4479      throws IOException {
4480    checkOperation(OperationCategory.READ);
4481    DirectoryListing dl = null;
4482    readLock();
4483    try {
4484      checkOperation(NameNode.OperationCategory.READ);
4485      dl = FSDirStatAndListingOp.getListingInt(dir, src, startAfter,
4486          needLocation);
4487    } catch (AccessControlException e) {
4488      logAuditEvent(false, "listStatus", src);
4489      throw e;
4490    } finally {
4491      readUnlock();
4492    }
4493    logAuditEvent(true, "listStatus", src);
4494    return dl;
4495  }
4496
4497  /////////////////////////////////////////////////////////
4498  //
4499  // These methods are called by datanodes
4500  //
4501  /////////////////////////////////////////////////////////
4502  /**
4503   * Register Datanode.
4504   * <p>
4505   * The purpose of registration is to identify whether the new datanode
4506   * serves a new data storage, and will report new data block copies,
4507   * which the namenode was not aware of; or the datanode is a replacement
4508   * node for the data storage that was previously served by a different
4509   * or the same (in terms of host:port) datanode.
4510   * The data storages are distinguished by their storageIDs. When a new
4511   * data storage is reported the namenode issues a new unique storageID.
4512   * <p>
4513   * Finally, the namenode returns its namespaceID as the registrationID
4514   * for the datanodes. 
4515   * namespaceID is a persistent attribute of the name space.
4516   * The registrationID is checked every time the datanode is communicating
4517   * with the namenode. 
4518   * Datanodes with inappropriate registrationID are rejected.
4519   * If the namenode stops, and then restarts it can restore its 
4520   * namespaceID and will continue serving the datanodes that has previously
4521   * registered with the namenode without restarting the whole cluster.
4522   * 
4523   * @see org.apache.hadoop.hdfs.server.datanode.DataNode
4524   */
4525  void registerDatanode(DatanodeRegistration nodeReg) throws IOException {
4526    writeLock();
4527    try {
4528      getBlockManager().getDatanodeManager().registerDatanode(nodeReg);
4529      checkSafeMode();
4530    } finally {
4531      writeUnlock();
4532    }
4533  }
4534  
4535  /**
4536   * Get registrationID for datanodes based on the namespaceID.
4537   * 
4538   * @see #registerDatanode(DatanodeRegistration)
4539   * @return registration ID
4540   */
4541  String getRegistrationID() {
4542    return Storage.getRegistrationID(getFSImage().getStorage());
4543  }
4544
4545  /**
4546   * The given node has reported in.  This method should:
4547   * 1) Record the heartbeat, so the datanode isn't timed out
4548   * 2) Adjust usage stats for future block allocation
4549   * 
4550   * If a substantial amount of time passed since the last datanode 
4551   * heartbeat then request an immediate block report.  
4552   * 
4553   * @return an array of datanode commands 
4554   * @throws IOException
4555   */
4556  HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
4557      StorageReport[] reports, long cacheCapacity, long cacheUsed,
4558      int xceiverCount, int xmitsInProgress, int failedVolumes,
4559      VolumeFailureSummary volumeFailureSummary) throws IOException {
4560    readLock();
4561    try {
4562      //get datanode commands
4563      final int maxTransfer = blockManager.getMaxReplicationStreams()
4564          - xmitsInProgress;
4565      DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
4566          nodeReg, reports, blockPoolId, cacheCapacity, cacheUsed,
4567          xceiverCount, maxTransfer, failedVolumes, volumeFailureSummary);
4568      
4569      //create ha status
4570      final NNHAStatusHeartbeat haState = new NNHAStatusHeartbeat(
4571          haContext.getState().getServiceState(),
4572          getFSImage().getLastAppliedOrWrittenTxId());
4573
4574      return new HeartbeatResponse(cmds, haState, rollingUpgradeInfo);
4575    } finally {
4576      readUnlock();
4577    }
4578  }
4579
4580  /**
4581   * Returns whether or not there were available resources at the last check of
4582   * resources.
4583   *
4584   * @return true if there were sufficient resources available, false otherwise.
4585   */
4586  boolean nameNodeHasResourcesAvailable() {
4587    return hasResourcesAvailable;
4588  }
4589
4590  /**
4591   * Perform resource checks and cache the results.
4592   */
4593  void checkAvailableResources() {
4594    Preconditions.checkState(nnResourceChecker != null,
4595        "nnResourceChecker not initialized");
4596    hasResourcesAvailable = nnResourceChecker.hasAvailableDiskSpace();
4597  }
4598
4599  /**
4600   * Persist the block list for the inode.
4601   * @param path
4602   * @param file
4603   * @param logRetryCache
4604   */
4605  private void persistBlocks(String path, INodeFile file,
4606                             boolean logRetryCache) {
4607    assert hasWriteLock();
4608    Preconditions.checkArgument(file.isUnderConstruction());
4609    getEditLog().logUpdateBlocks(path, file, logRetryCache);
4610    NameNode.stateChangeLog.debug("persistBlocks: {} with {} blocks is" +
4611        " peristed to the file system", path, file.getBlocks().length);
4612  }
4613
4614  /**
4615   * Close file.
4616   * @param path
4617   * @param file
4618   */
4619  private void closeFile(String path, INodeFile file) {
4620    assert hasWriteLock();
4621    waitForLoadingFSImage();
4622    // file is closed
4623    getEditLog().logCloseFile(path, file);
4624    NameNode.stateChangeLog.debug("closeFile: {} with {} blocks is persisted" +
4625        " to the file system", path, file.getBlocks().length);
4626  }
4627
4628  /**
4629   * Periodically calls hasAvailableResources of NameNodeResourceChecker, and if
4630   * there are found to be insufficient resources available, causes the NN to
4631   * enter safe mode. If resources are later found to have returned to
4632   * acceptable levels, this daemon will cause the NN to exit safe mode.
4633   */
4634  class NameNodeResourceMonitor implements Runnable  {
4635    boolean shouldNNRmRun = true;
4636    @Override
4637    public void run () {
4638      try {
4639        while (fsRunning && shouldNNRmRun) {
4640          checkAvailableResources();
4641          if(!nameNodeHasResourcesAvailable()) {
4642            String lowResourcesMsg = "NameNode low on available disk space. ";
4643            if (!isInSafeMode()) {
4644              LOG.warn(lowResourcesMsg + "Entering safe mode.");
4645            } else {
4646              LOG.warn(lowResourcesMsg + "Already in safe mode.");
4647            }
4648            enterSafeMode(true);
4649          }
4650          try {
4651            Thread.sleep(resourceRecheckInterval);
4652          } catch (InterruptedException ie) {
4653            // Deliberately ignore
4654          }
4655        }
4656      } catch (Exception e) {
4657        FSNamesystem.LOG.error("Exception in NameNodeResourceMonitor: ", e);
4658      }
4659    }
4660
4661    public void stopMonitor() {
4662      shouldNNRmRun = false;
4663    }
4664 }
4665
4666  class NameNodeEditLogRoller implements Runnable {
4667
4668    private boolean shouldRun = true;
4669    private final long rollThreshold;
4670    private final long sleepIntervalMs;
4671
4672    public NameNodeEditLogRoller(long rollThreshold, int sleepIntervalMs) {
4673        this.rollThreshold = rollThreshold;
4674        this.sleepIntervalMs = sleepIntervalMs;
4675    }
4676
4677    @Override
4678    public void run() {
4679      while (fsRunning && shouldRun) {
4680        try {
4681          FSEditLog editLog = getFSImage().getEditLog();
4682          long numEdits =
4683              editLog.getLastWrittenTxId() - editLog.getCurSegmentTxId();
4684          if (numEdits > rollThreshold) {
4685            FSNamesystem.LOG.info("NameNode rolling its own edit log because"
4686                + " number of edits in open segment exceeds threshold of "
4687                + rollThreshold);
4688            rollEditLog();
4689          }
4690        } catch (Exception e) {
4691          FSNamesystem.LOG.error("Swallowing exception in "
4692              + NameNodeEditLogRoller.class.getSimpleName() + ":", e);
4693        }
4694        try {
4695          Thread.sleep(sleepIntervalMs);
4696        } catch (InterruptedException e) {
4697          FSNamesystem.LOG.info(NameNodeEditLogRoller.class.getSimpleName()
4698              + " was interrupted, exiting");
4699          break;
4700        }
4701      }
4702    }
4703
4704    public void stop() {
4705      shouldRun = false;
4706    }
4707  }
4708
4709  /**
4710   * Daemon to periodically scan the namespace for lazyPersist files
4711   * with missing blocks and unlink them.
4712   */
4713  class LazyPersistFileScrubber implements Runnable {
4714    private volatile boolean shouldRun = true;
4715    final int scrubIntervalSec;
4716    public LazyPersistFileScrubber(final int scrubIntervalSec) {
4717      this.scrubIntervalSec = scrubIntervalSec;
4718    }
4719
4720    /**
4721     * Periodically go over the list of lazyPersist files with missing
4722     * blocks and unlink them from the namespace.
4723     */
4724    private void clearCorruptLazyPersistFiles()
4725        throws IOException {
4726
4727      BlockStoragePolicy lpPolicy = blockManager.getStoragePolicy("LAZY_PERSIST");
4728
4729      List<BlockCollection> filesToDelete = new ArrayList<>();
4730      boolean changed = false;
4731      writeLock();
4732      try {
4733        final Iterator<Block> it = blockManager.getCorruptReplicaBlockIterator();
4734
4735        while (it.hasNext()) {
4736          Block b = it.next();
4737          BlockInfoContiguous blockInfo = blockManager.getStoredBlock(b);
4738          if (blockInfo.getBlockCollection().getStoragePolicyID()
4739              == lpPolicy.getId()) {
4740            filesToDelete.add(blockInfo.getBlockCollection());
4741          }
4742        }
4743
4744        for (BlockCollection bc : filesToDelete) {
4745          LOG.warn("Removing lazyPersist file " + bc.getName() + " with no replicas.");
4746          BlocksMapUpdateInfo toRemoveBlocks =
4747              FSDirDeleteOp.deleteInternal(
4748                  FSNamesystem.this, bc.getName(),
4749                  INodesInPath.fromINode((INodeFile) bc), false);
4750          changed |= toRemoveBlocks != null;
4751          if (toRemoveBlocks != null) {
4752            removeBlocks(toRemoveBlocks); // Incremental deletion of blocks
4753          }
4754        }
4755      } finally {
4756        writeUnlock();
4757      }
4758      if (changed) {
4759        getEditLog().logSync();
4760      }
4761    }
4762
4763    @Override
4764    public void run() {
4765      while (fsRunning && shouldRun) {
4766        try {
4767          clearCorruptLazyPersistFiles();
4768          Thread.sleep(scrubIntervalSec * 1000);
4769        } catch (InterruptedException e) {
4770          FSNamesystem.LOG.info(
4771              "LazyPersistFileScrubber was interrupted, exiting");
4772          break;
4773        } catch (Exception e) {
4774          FSNamesystem.LOG.error(
4775              "Ignoring exception in LazyPersistFileScrubber:", e);
4776        }
4777      }
4778    }
4779
4780    public void stop() {
4781      shouldRun = false;
4782    }
4783  }
4784
4785  public FSImage getFSImage() {
4786    return fsImage;
4787  }
4788
4789  public FSEditLog getEditLog() {
4790    return getFSImage().getEditLog();
4791  }    
4792
4793  private void checkBlock(ExtendedBlock block) throws IOException {
4794    if (block != null && !this.blockPoolId.equals(block.getBlockPoolId())) {
4795      throw new IOException("Unexpected BlockPoolId " + block.getBlockPoolId()
4796          + " - expected " + blockPoolId);
4797    }
4798  }
4799
4800  @Metric({"MissingBlocks", "Number of missing blocks"})
4801  public long getMissingBlocksCount() {
4802    // not locking
4803    return blockManager.getMissingBlocksCount();
4804  }
4805
4806  @Metric({"MissingReplOneBlocks", "Number of missing blocks " +
4807      "with replication factor 1"})
4808  public long getMissingReplOneBlocksCount() {
4809    // not locking
4810    return blockManager.getMissingReplOneBlocksCount();
4811  }
4812  
4813  @Metric({"ExpiredHeartbeats", "Number of expired heartbeats"})
4814  public int getExpiredHeartbeats() {
4815    return datanodeStatistics.getExpiredHeartbeats();
4816  }
4817  
4818  @Metric({"TransactionsSinceLastCheckpoint",
4819      "Number of transactions since last checkpoint"})
4820  public long getTransactionsSinceLastCheckpoint() {
4821    return getEditLog().getLastWrittenTxId() -
4822        getFSImage().getStorage().getMostRecentCheckpointTxId();
4823  }
4824  
4825  @Metric({"TransactionsSinceLastLogRoll",
4826      "Number of transactions since last edit log roll"})
4827  public long getTransactionsSinceLastLogRoll() {
4828    if (isInStandbyState() || !getEditLog().isSegmentOpen()) {
4829      return 0;
4830    } else {
4831      return getEditLog().getLastWrittenTxId() -
4832        getEditLog().getCurSegmentTxId() + 1;
4833    }
4834  }
4835  
4836  @Metric({"LastWrittenTransactionId", "Transaction ID written to the edit log"})
4837  public long getLastWrittenTransactionId() {
4838    return getEditLog().getLastWrittenTxId();
4839  }
4840  
4841  @Metric({"LastCheckpointTime",
4842      "Time in milliseconds since the epoch of the last checkpoint"})
4843  public long getLastCheckpointTime() {
4844    return getFSImage().getStorage().getMostRecentCheckpointTime();
4845  }
4846
4847  /** @see ClientProtocol#getStats() */
4848  long[] getStats() {
4849    final long[] stats = datanodeStatistics.getStats();
4850    stats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = getUnderReplicatedBlocks();
4851    stats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = getCorruptReplicaBlocks();
4852    stats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = getMissingBlocksCount();
4853    stats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] =
4854        getMissingReplOneBlocksCount();
4855    return stats;
4856  }
4857
4858  @Override // FSNamesystemMBean
4859  @Metric({"CapacityTotal",
4860      "Total raw capacity of data nodes in bytes"})
4861  public long getCapacityTotal() {
4862    return datanodeStatistics.getCapacityTotal();
4863  }
4864
4865  @Metric({"CapacityTotalGB",
4866      "Total raw capacity of data nodes in GB"})
4867  public float getCapacityTotalGB() {
4868    return DFSUtil.roundBytesToGB(getCapacityTotal());
4869  }
4870
4871  @Override // FSNamesystemMBean
4872  @Metric({"CapacityUsed",
4873      "Total used capacity across all data nodes in bytes"})
4874  public long getCapacityUsed() {
4875    return datanodeStatistics.getCapacityUsed();
4876  }
4877
4878  @Metric({"CapacityUsedGB",
4879      "Total used capacity across all data nodes in GB"})
4880  public float getCapacityUsedGB() {
4881    return DFSUtil.roundBytesToGB(getCapacityUsed());
4882  }
4883
4884  @Override // FSNamesystemMBean
4885  @Metric({"CapacityRemaining", "Remaining capacity in bytes"})
4886  public long getCapacityRemaining() {
4887    return datanodeStatistics.getCapacityRemaining();
4888  }
4889
4890  @Metric({"CapacityRemainingGB", "Remaining capacity in GB"})
4891  public float getCapacityRemainingGB() {
4892    return DFSUtil.roundBytesToGB(getCapacityRemaining());
4893  }
4894
4895  @Metric({"CapacityUsedNonDFS",
4896      "Total space used by data nodes for non DFS purposes in bytes"})
4897  public long getCapacityUsedNonDFS() {
4898    return datanodeStatistics.getCapacityUsedNonDFS();
4899  }
4900
4901  /**
4902   * Total number of connections.
4903   */
4904  @Override // FSNamesystemMBean
4905  @Metric
4906  public int getTotalLoad() {
4907    return datanodeStatistics.getXceiverCount();
4908  }
4909  
4910  @Metric({ "SnapshottableDirectories", "Number of snapshottable directories" })
4911  public int getNumSnapshottableDirs() {
4912    return this.snapshotManager.getNumSnapshottableDirs();
4913  }
4914
4915  @Metric({ "Snapshots", "The number of snapshots" })
4916  public int getNumSnapshots() {
4917    return this.snapshotManager.getNumSnapshots();
4918  }
4919
4920  @Override
4921  public String getSnapshotStats() {
4922    Map<String, Object> info = new HashMap<String, Object>();
4923    info.put("SnapshottableDirectories", this.getNumSnapshottableDirs());
4924    info.put("Snapshots", this.getNumSnapshots());
4925    return JSON.toString(info);
4926  }
4927
4928  int getNumberOfDatanodes(DatanodeReportType type) {
4929    readLock();
4930    try {
4931      return getBlockManager().getDatanodeManager().getDatanodeListForReport(
4932          type).size(); 
4933    } finally {
4934      readUnlock();
4935    }
4936  }
4937
4938  DatanodeInfo[] datanodeReport(final DatanodeReportType type
4939      ) throws AccessControlException, StandbyException {
4940    checkSuperuserPrivilege();
4941    checkOperation(OperationCategory.UNCHECKED);
4942    readLock();
4943    try {
4944      checkOperation(OperationCategory.UNCHECKED);
4945      final DatanodeManager dm = getBlockManager().getDatanodeManager();      
4946      final List<DatanodeDescriptor> results = dm.getDatanodeListForReport(type);
4947
4948      DatanodeInfo[] arr = new DatanodeInfo[results.size()];
4949      for (int i=0; i<arr.length; i++) {
4950        arr[i] = new DatanodeInfo(results.get(i));
4951      }
4952      return arr;
4953    } finally {
4954      readUnlock();
4955    }
4956  }
4957
4958  DatanodeStorageReport[] getDatanodeStorageReport(final DatanodeReportType type
4959      ) throws AccessControlException, StandbyException {
4960    checkSuperuserPrivilege();
4961    checkOperation(OperationCategory.UNCHECKED);
4962    readLock();
4963    try {
4964      checkOperation(OperationCategory.UNCHECKED);
4965      final DatanodeManager dm = getBlockManager().getDatanodeManager();      
4966      final List<DatanodeDescriptor> datanodes = dm.getDatanodeListForReport(type);
4967
4968      DatanodeStorageReport[] reports = new DatanodeStorageReport[datanodes.size()];
4969      for (int i = 0; i < reports.length; i++) {
4970        final DatanodeDescriptor d = datanodes.get(i);
4971        reports[i] = new DatanodeStorageReport(new DatanodeInfo(d),
4972            d.getStorageReports());
4973      }
4974      return reports;
4975    } finally {
4976      readUnlock();
4977    }
4978  }
4979
4980  /**
4981   * Save namespace image.
4982   * This will save current namespace into fsimage file and empty edits file.
4983   * Requires superuser privilege and safe mode.
4984   * 
4985   * @throws AccessControlException if superuser privilege is violated.
4986   * @throws IOException if 
4987   */
4988  void saveNamespace() throws AccessControlException, IOException {
4989    checkOperation(OperationCategory.UNCHECKED);
4990    checkSuperuserPrivilege();
4991
4992    cpLock();  // Block if a checkpointing is in progress on standby.
4993    readLock();
4994    try {
4995      checkOperation(OperationCategory.UNCHECKED);
4996
4997      if (!isInSafeMode()) {
4998        throw new IOException("Safe mode should be turned ON "
4999            + "in order to create namespace image.");
5000      }
5001      getFSImage().saveNamespace(this);
5002    } finally {
5003      readUnlock();
5004      cpUnlock();
5005    }
5006    LOG.info("New namespace image has been created");
5007  }
5008  
5009  /**
5010   * Enables/Disables/Checks restoring failed storage replicas if the storage becomes available again.
5011   * Requires superuser privilege.
5012   * 
5013   * @throws AccessControlException if superuser privilege is violated.
5014   */
5015  boolean restoreFailedStorage(String arg) throws AccessControlException,
5016      StandbyException {
5017    checkSuperuserPrivilege();
5018    checkOperation(OperationCategory.UNCHECKED);
5019    cpLock();  // Block if a checkpointing is in progress on standby.
5020    writeLock();
5021    try {
5022      checkOperation(OperationCategory.UNCHECKED);
5023      
5024      // if it is disabled - enable it and vice versa.
5025      if(arg.equals("check"))
5026        return getFSImage().getStorage().getRestoreFailedStorage();
5027      
5028      boolean val = arg.equals("true");  // false if not
5029      getFSImage().getStorage().setRestoreFailedStorage(val);
5030      
5031      return val;
5032    } finally {
5033      writeUnlock();
5034      cpUnlock();
5035    }
5036  }
5037
5038  Date getStartTime() {
5039    return new Date(startTime); 
5040  }
5041    
5042  void finalizeUpgrade() throws IOException {
5043    checkSuperuserPrivilege();
5044    checkOperation(OperationCategory.UNCHECKED);
5045    cpLock();  // Block if a checkpointing is in progress on standby.
5046    writeLock();
5047    try {
5048      checkOperation(OperationCategory.UNCHECKED);
5049      getFSImage().finalizeUpgrade(this.isHaEnabled() && inActiveState());
5050    } finally {
5051      writeUnlock();
5052      cpUnlock();
5053    }
5054  }
5055
5056  void refreshNodes() throws IOException {
5057    checkOperation(OperationCategory.UNCHECKED);
5058    checkSuperuserPrivilege();
5059    getBlockManager().getDatanodeManager().refreshNodes(new HdfsConfiguration());
5060  }
5061
5062  void setBalancerBandwidth(long bandwidth) throws IOException {
5063    checkOperation(OperationCategory.UNCHECKED);
5064    checkSuperuserPrivilege();
5065    getBlockManager().getDatanodeManager().setBalancerBandwidth(bandwidth);
5066  }
5067
5068  /**
5069   * Persist the new block (the last block of the given file).
5070   * @param path
5071   * @param file
5072   */
5073  private void persistNewBlock(String path, INodeFile file) {
5074    Preconditions.checkArgument(file.isUnderConstruction());
5075    getEditLog().logAddBlock(path, file);
5076    NameNode.stateChangeLog.debug("persistNewBlock: {} with new block {}," +
5077        " current total block count is {}", path,
5078        file.getLastBlock().toString(), file.getBlocks().length);
5079  }
5080
5081  /**
5082   * SafeModeInfo contains information related to the safe mode.
5083   * <p>
5084   * An instance of {@link SafeModeInfo} is created when the name node
5085   * enters safe mode.
5086   * <p>
5087   * During name node startup {@link SafeModeInfo} counts the number of
5088   * <em>safe blocks</em>, those that have at least the minimal number of
5089   * replicas, and calculates the ratio of safe blocks to the total number
5090   * of blocks in the system, which is the size of blocks in
5091   * {@link FSNamesystem#blockManager}. When the ratio reaches the
5092   * {@link #threshold} it starts the SafeModeMonitor daemon in order
5093   * to monitor whether the safe mode {@link #extension} is passed.
5094   * Then it leaves safe mode and destroys itself.
5095   * <p>
5096   * If safe mode is turned on manually then the number of safe blocks is
5097   * not tracked because the name node is not intended to leave safe mode
5098   * automatically in the case.
5099   *
5100   * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction, boolean)
5101   */
5102  public class SafeModeInfo {
5103    // configuration fields
5104    /** Safe mode threshold condition %.*/
5105    private final double threshold;
5106    /** Safe mode minimum number of datanodes alive */
5107    private final int datanodeThreshold;
5108    /**
5109     * Safe mode extension after the threshold.
5110     * Make it volatile so that getSafeModeTip can read the latest value
5111     * without taking a lock.
5112     */
5113    private volatile int extension;
5114    /** Min replication required by safe mode. */
5115    private final int safeReplication;
5116    /** threshold for populating needed replication queues */
5117    private final double replQueueThreshold;
5118    // internal fields
5119    /** Time when threshold was reached.
5120     * <br> -1 safe mode is off
5121     * <br> 0 safe mode is on, and threshold is not reached yet
5122     * <br> >0 safe mode is on, but we are in extension period 
5123     */
5124    private long reached = -1;  
5125    private long reachedTimestamp = -1;
5126    /** Total number of blocks. */
5127    int blockTotal; 
5128    /** Number of safe blocks. */
5129    int blockSafe;
5130    /** Number of blocks needed to satisfy safe mode threshold condition */
5131    private int blockThreshold;
5132    /** Number of blocks needed before populating replication queues */
5133    private int blockReplQueueThreshold;
5134    /** time of the last status printout */
5135    private long lastStatusReport = 0;
5136    /**
5137     * Was safemode entered automatically because available resources were low.
5138     * Make it volatile so that getSafeModeTip can read the latest value
5139     * without taking a lock.
5140     */
5141    private volatile boolean resourcesLow = false;
5142    /** Should safemode adjust its block totals as blocks come in */
5143    private boolean shouldIncrementallyTrackBlocks = false;
5144    /** counter for tracking startup progress of reported blocks */
5145    private Counter awaitingReportedBlocksCounter;
5146    
5147    /**
5148     * Creates SafeModeInfo when the name node enters
5149     * automatic safe mode at startup.
5150     *  
5151     * @param conf configuration
5152     */
5153    private SafeModeInfo(Configuration conf) {
5154      this.threshold = conf.getFloat(DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY,
5155          DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_DEFAULT);
5156      if(threshold > 1.0) {
5157        LOG.warn("The threshold value should't be greater than 1, threshold: " + threshold);
5158      }
5159      this.datanodeThreshold = conf.getInt(
5160        DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY,
5161        DFS_NAMENODE_SAFEMODE_MIN_DATANODES_DEFAULT);
5162      this.extension = conf.getInt(DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0);
5163      this.safeReplication = conf.getInt(DFS_NAMENODE_REPLICATION_MIN_KEY, 
5164                                         DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
5165      
5166      LOG.info(DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY + " = " + threshold);
5167      LOG.info(DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY + " = " + datanodeThreshold);
5168      LOG.info(DFS_NAMENODE_SAFEMODE_EXTENSION_KEY + "     = " + extension);
5169
5170      // default to safe mode threshold (i.e., don't populate queues before leaving safe mode)
5171      this.replQueueThreshold = 
5172        conf.getFloat(DFS_NAMENODE_REPL_QUEUE_THRESHOLD_PCT_KEY,
5173                      (float) threshold);
5174      this.blockTotal = 0; 
5175      this.blockSafe = 0;
5176    }
5177
5178    /**
5179     * In the HA case, the StandbyNode can be in safemode while the namespace
5180     * is modified by the edit log tailer. In this case, the number of total
5181     * blocks changes as edits are processed (eg blocks are added and deleted).
5182     * However, we don't want to do the incremental tracking during the
5183     * startup-time loading process -- only once the initial total has been
5184     * set after the image has been loaded.
5185     */
5186    private boolean shouldIncrementallyTrackBlocks() {
5187      return shouldIncrementallyTrackBlocks;
5188    }
5189
5190    /**
5191     * Creates SafeModeInfo when safe mode is entered manually, or because
5192     * available resources are low.
5193     *
5194     * The {@link #threshold} is set to 1.5 so that it could never be reached.
5195     * {@link #blockTotal} is set to -1 to indicate that safe mode is manual.
5196     * 
5197     * @see SafeModeInfo
5198     */
5199    private SafeModeInfo(boolean resourcesLow) {
5200      this.threshold = 1.5f;  // this threshold can never be reached
5201      this.datanodeThreshold = Integer.MAX_VALUE;
5202      this.extension = Integer.MAX_VALUE;
5203      this.safeReplication = Short.MAX_VALUE + 1; // more than maxReplication
5204      this.replQueueThreshold = 1.5f; // can never be reached
5205      this.blockTotal = -1;
5206      this.blockSafe = -1;
5207      this.resourcesLow = resourcesLow;
5208      enter();
5209      reportStatus("STATE* Safe mode is ON.", true);
5210    }
5211      
5212    /**
5213     * Check if safe mode is on.
5214     * @return true if in safe mode
5215     */
5216    private synchronized boolean isOn() {
5217      doConsistencyCheck();
5218      return this.reached >= 0;
5219    }
5220      
5221    /**
5222     * Enter safe mode.
5223     */
5224    private void enter() {
5225      this.reached = 0;
5226      this.reachedTimestamp = 0;
5227    }
5228      
5229    /**
5230     * Leave safe mode.
5231     * <p>
5232     * Check for invalid, under- & over-replicated blocks in the end of startup.
5233     */
5234    private synchronized void leave() {
5235      // if not done yet, initialize replication queues.
5236      // In the standby, do not populate repl queues
5237      if (!isPopulatingReplQueues() && shouldPopulateReplQueues()) {
5238        initializeReplQueues();
5239      }
5240      long timeInSafemode = now() - startTime;
5241      NameNode.stateChangeLog.info("STATE* Leaving safe mode after " 
5242                                    + timeInSafemode/1000 + " secs");
5243      NameNode.getNameNodeMetrics().setSafeModeTime((int) timeInSafemode);
5244
5245      //Log the following only once (when transitioning from ON -> OFF)
5246      if (reached >= 0) {
5247        NameNode.stateChangeLog.info("STATE* Safe mode is OFF"); 
5248      }
5249      reached = -1;
5250      reachedTimestamp = -1;
5251      safeMode = null;
5252      final NetworkTopology nt = blockManager.getDatanodeManager().getNetworkTopology();
5253      NameNode.stateChangeLog.info("STATE* Network topology has "
5254          + nt.getNumOfRacks() + " racks and "
5255          + nt.getNumOfLeaves() + " datanodes");
5256      NameNode.stateChangeLog.info("STATE* UnderReplicatedBlocks has "
5257          + blockManager.numOfUnderReplicatedBlocks() + " blocks");
5258
5259      startSecretManagerIfNecessary();
5260
5261      // If startup has not yet completed, end safemode phase.
5262      StartupProgress prog = NameNode.getStartupProgress();
5263      if (prog.getStatus(Phase.SAFEMODE) != Status.COMPLETE) {
5264        prog.endStep(Phase.SAFEMODE, STEP_AWAITING_REPORTED_BLOCKS);
5265        prog.endPhase(Phase.SAFEMODE);
5266      }
5267    }
5268
5269    /**
5270     * Check whether we have reached the threshold for 
5271     * initializing replication queues.
5272     */
5273    private synchronized boolean canInitializeReplQueues() {
5274      return shouldPopulateReplQueues()
5275          && blockSafe >= blockReplQueueThreshold;
5276    }
5277      
5278    /** 
5279     * Safe mode can be turned off iff 
5280     * the threshold is reached and 
5281     * the extension time have passed.
5282     * @return true if can leave or false otherwise.
5283     */
5284    private synchronized boolean canLeave() {
5285      if (reached == 0) {
5286        return false;
5287      }
5288
5289      if (monotonicNow() - reached < extension) {
5290        reportStatus("STATE* Safe mode ON, in safe mode extension.", false);
5291        return false;
5292      }
5293
5294      if (needEnter()) {
5295        reportStatus("STATE* Safe mode ON, thresholds not met.", false);
5296        return false;
5297      }
5298
5299      return true;
5300    }
5301      
5302    /** 
5303     * There is no need to enter safe mode 
5304     * if DFS is empty or {@link #threshold} == 0
5305     */
5306    private boolean needEnter() {
5307      return (threshold != 0 && blockSafe < blockThreshold) ||
5308        (datanodeThreshold != 0 && getNumLiveDataNodes() < datanodeThreshold) ||
5309        (!nameNodeHasResourcesAvailable());
5310    }
5311      
5312    /**
5313     * Check and trigger safe mode if needed. 
5314     */
5315    private void checkMode() {
5316      // Have to have write-lock since leaving safemode initializes
5317      // repl queues, which requires write lock
5318      assert hasWriteLock();
5319      if (inTransitionToActive()) {
5320        return;
5321      }
5322      // if smmthread is already running, the block threshold must have been 
5323      // reached before, there is no need to enter the safe mode again
5324      if (smmthread == null && needEnter()) {
5325        enter();
5326        // check if we are ready to initialize replication queues
5327        if (canInitializeReplQueues() && !isPopulatingReplQueues()
5328            && !haEnabled) {
5329          initializeReplQueues();
5330        }
5331        reportStatus("STATE* Safe mode ON.", false);
5332        return;
5333      }
5334      // the threshold is reached or was reached before
5335      if (!isOn() ||                           // safe mode is off
5336          extension <= 0 || threshold <= 0) {  // don't need to wait
5337        this.leave(); // leave safe mode
5338        return;
5339      }
5340      if (reached > 0) {  // threshold has already been reached before
5341        reportStatus("STATE* Safe mode ON.", false);
5342        return;
5343      }
5344      // start monitor
5345      reached = monotonicNow();
5346      reachedTimestamp = now();
5347      if (smmthread == null) {
5348        smmthread = new Daemon(new SafeModeMonitor());
5349        smmthread.start();
5350        reportStatus("STATE* Safe mode extension entered.", true);
5351      }
5352
5353      // check if we are ready to initialize replication queues
5354      if (canInitializeReplQueues() && !isPopulatingReplQueues() && !haEnabled) {
5355        initializeReplQueues();
5356      }
5357    }
5358      
5359    /**
5360     * Set total number of blocks.
5361     */
5362    private synchronized void setBlockTotal(int total) {
5363      this.blockTotal = total;
5364      this.blockThreshold = (int) (blockTotal * threshold);
5365      this.blockReplQueueThreshold = 
5366        (int) (blockTotal * replQueueThreshold);
5367      if (haEnabled) {
5368        // After we initialize the block count, any further namespace
5369        // modifications done while in safe mode need to keep track
5370        // of the number of total blocks in the system.
5371        this.shouldIncrementallyTrackBlocks = true;
5372      }
5373      if(blockSafe < 0)
5374        this.blockSafe = 0;
5375      checkMode();
5376    }
5377      
5378    /**
5379     * Increment number of safe blocks if current block has 
5380     * reached minimal replication.
5381     * @param replication current replication 
5382     */
5383    private synchronized void incrementSafeBlockCount(short replication) {
5384      if (replication == safeReplication) {
5385        this.blockSafe++;
5386
5387        // Report startup progress only if we haven't completed startup yet.
5388        StartupProgress prog = NameNode.getStartupProgress();
5389        if (prog.getStatus(Phase.SAFEMODE) != Status.COMPLETE) {
5390          if (this.awaitingReportedBlocksCounter == null) {
5391            this.awaitingReportedBlocksCounter = prog.getCounter(Phase.SAFEMODE,
5392              STEP_AWAITING_REPORTED_BLOCKS);
5393          }
5394          this.awaitingReportedBlocksCounter.increment();
5395        }
5396
5397        checkMode();
5398      }
5399    }
5400      
5401    /**
5402     * Decrement number of safe blocks if current block has 
5403     * fallen below minimal replication.
5404     * @param replication current replication 
5405     */
5406    private synchronized void decrementSafeBlockCount(short replication) {
5407      if (replication == safeReplication-1) {
5408        this.blockSafe--;
5409        //blockSafe is set to -1 in manual / low resources safemode
5410        assert blockSafe >= 0 || isManual() || areResourcesLow();
5411        checkMode();
5412      }
5413    }
5414
5415    /**
5416     * Check if safe mode was entered manually
5417     */
5418    private boolean isManual() {
5419      return extension == Integer.MAX_VALUE;
5420    }
5421
5422    /**
5423     * Set manual safe mode.
5424     */
5425    private synchronized void setManual() {
5426      extension = Integer.MAX_VALUE;
5427    }
5428
5429    /**
5430     * Check if safe mode was entered due to resources being low.
5431     */
5432    private boolean areResourcesLow() {
5433      return resourcesLow;
5434    }
5435
5436    /**
5437     * Set that resources are low for this instance of safe mode.
5438     */
5439    private void setResourcesLow() {
5440      resourcesLow = true;
5441    }
5442
5443    /**
5444     * A tip on how safe mode is to be turned off: manually or automatically.
5445     */
5446    String getTurnOffTip() {
5447      if(!isOn()) {
5448        return "Safe mode is OFF.";
5449      }
5450
5451      //Manual OR low-resource safemode. (Admin intervention required)
5452      String adminMsg = "It was turned on manually. ";
5453      if (areResourcesLow()) {
5454        adminMsg = "Resources are low on NN. Please add or free up more "
5455          + "resources then turn off safe mode manually. NOTE:  If you turn off"
5456          + " safe mode before adding resources, "
5457          + "the NN will immediately return to safe mode. ";
5458      }
5459      if (isManual() || areResourcesLow()) {
5460        return adminMsg
5461          + "Use \"hdfs dfsadmin -safemode leave\" to turn safe mode off.";
5462      }
5463
5464      boolean thresholdsMet = true;
5465      int numLive = getNumLiveDataNodes();
5466      String msg = "";
5467      if (blockSafe < blockThreshold) {
5468        msg += String.format(
5469          "The reported blocks %d needs additional %d"
5470          + " blocks to reach the threshold %.4f of total blocks %d.%n",
5471          blockSafe, (blockThreshold - blockSafe) + 1, threshold, blockTotal);
5472        thresholdsMet = false;
5473      } else {
5474        msg += String.format("The reported blocks %d has reached the threshold"
5475            + " %.4f of total blocks %d. ", blockSafe, threshold, blockTotal);
5476      }
5477      if (numLive < datanodeThreshold) {
5478        msg += String.format(
5479          "The number of live datanodes %d needs an additional %d live "
5480          + "datanodes to reach the minimum number %d.%n",
5481          numLive, (datanodeThreshold - numLive), datanodeThreshold);
5482        thresholdsMet = false;
5483      } else {
5484        msg += String.format("The number of live datanodes %d has reached "
5485            + "the minimum number %d. ",
5486            numLive, datanodeThreshold);
5487      }
5488      msg += (reached > 0) ? "In safe mode extension. " : "";
5489      msg += "Safe mode will be turned off automatically ";
5490
5491      if (!thresholdsMet) {
5492        msg += "once the thresholds have been reached.";
5493      } else if (reached + extension - monotonicNow() > 0) {
5494        msg += ("in " + (reached + extension - monotonicNow()) / 1000 + " seconds.");
5495      } else {
5496        msg += "soon.";
5497      }
5498
5499      return msg;
5500    }
5501
5502    /**
5503     * Print status every 20 seconds.
5504     */
5505    private void reportStatus(String msg, boolean rightNow) {
5506      long curTime = now();
5507      if(!rightNow && (curTime - lastStatusReport < 20 * 1000))
5508        return;
5509      NameNode.stateChangeLog.info(msg + " \n" + getTurnOffTip());
5510      lastStatusReport = curTime;
5511    }
5512
5513    @Override
5514    public String toString() {
5515      String resText = "Current safe blocks = " 
5516        + blockSafe 
5517        + ". Target blocks = " + blockThreshold + " for threshold = %" + threshold
5518        + ". Minimal replication = " + safeReplication + ".";
5519      if (reached > 0) 
5520        resText += " Threshold was reached " + new Date(reachedTimestamp) + ".";
5521      return resText;
5522    }
5523      
5524    /**
5525     * Checks consistency of the class state.
5526     * This is costly so only runs if asserts are enabled.
5527     */
5528    private void doConsistencyCheck() {
5529      boolean assertsOn = false;
5530      assert assertsOn = true; // set to true if asserts are on
5531      if (!assertsOn) return;
5532      
5533      if (blockTotal == -1 && blockSafe == -1) {
5534        return; // manual safe mode
5535      }
5536      int activeBlocks = blockManager.getActiveBlockCount();
5537      if ((blockTotal != activeBlocks) &&
5538          !(blockSafe >= 0 && blockSafe <= blockTotal)) {
5539        throw new AssertionError(
5540            " SafeMode: Inconsistent filesystem state: "
5541        + "SafeMode data: blockTotal=" + blockTotal
5542        + " blockSafe=" + blockSafe + "; "
5543        + "BlockManager data: active="  + activeBlocks);
5544      }
5545    }
5546
5547    private synchronized void adjustBlockTotals(int deltaSafe, int deltaTotal) {
5548      if (!shouldIncrementallyTrackBlocks) {
5549        return;
5550      }
5551      assert haEnabled;
5552      
5553      if (LOG.isDebugEnabled()) {
5554        LOG.debug("Adjusting block totals from " +
5555            blockSafe + "/" + blockTotal + " to " +
5556            (blockSafe + deltaSafe) + "/" + (blockTotal + deltaTotal));
5557      }
5558      assert blockSafe + deltaSafe >= 0 : "Can't reduce blockSafe " +
5559        blockSafe + " by " + deltaSafe + ": would be negative";
5560      assert blockTotal + deltaTotal >= 0 : "Can't reduce blockTotal " +
5561        blockTotal + " by " + deltaTotal + ": would be negative";
5562      
5563      blockSafe += deltaSafe;
5564      setBlockTotal(blockTotal + deltaTotal);
5565    }
5566  }
5567    
5568  /**
5569   * Periodically check whether it is time to leave safe mode.
5570   * This thread starts when the threshold level is reached.
5571   *
5572   */
5573  class SafeModeMonitor implements Runnable {
5574    /** interval in msec for checking safe mode: {@value} */
5575    private static final long recheckInterval = 1000;
5576      
5577    /**
5578     */
5579    @Override
5580    public void run() {
5581      while (fsRunning) {
5582        writeLock();
5583        try {
5584          if (safeMode == null) { // Not in safe mode.
5585            break;
5586          }
5587          if (safeMode.canLeave()) {
5588            // Leave safe mode.
5589            safeMode.leave();
5590            smmthread = null;
5591            break;
5592          }
5593        } finally {
5594          writeUnlock();
5595        }
5596
5597        try {
5598          Thread.sleep(recheckInterval);
5599        } catch (InterruptedException ie) {
5600          // Ignored
5601        }
5602      }
5603      if (!fsRunning) {
5604        LOG.info("NameNode is being shutdown, exit SafeModeMonitor thread");
5605      }
5606    }
5607  }
5608    
5609  boolean setSafeMode(SafeModeAction action) throws IOException {
5610    if (action != SafeModeAction.SAFEMODE_GET) {
5611      checkSuperuserPrivilege();
5612      switch(action) {
5613      case SAFEMODE_LEAVE: // leave safe mode
5614        leaveSafeMode();
5615        break;
5616      case SAFEMODE_ENTER: // enter safe mode
5617        enterSafeMode(false);
5618        break;
5619      default:
5620        LOG.error("Unexpected safe mode action");
5621      }
5622    }
5623    return isInSafeMode();
5624  }
5625
5626  @Override
5627  public void checkSafeMode() {
5628    // safeMode is volatile, and may be set to null at any time
5629    SafeModeInfo safeMode = this.safeMode;
5630    if (safeMode != null) {
5631      safeMode.checkMode();
5632    }
5633  }
5634
5635  @Override
5636  public boolean isInSafeMode() {
5637    // safeMode is volatile, and may be set to null at any time
5638    SafeModeInfo safeMode = this.safeMode;
5639    if (safeMode == null)
5640      return false;
5641    return safeMode.isOn();
5642  }
5643
5644  @Override
5645  public boolean isInStartupSafeMode() {
5646    // safeMode is volatile, and may be set to null at any time
5647    SafeModeInfo safeMode = this.safeMode;
5648    if (safeMode == null)
5649      return false;
5650    // If the NN is in safemode, and not due to manual / low resources, we
5651    // assume it must be because of startup. If the NN had low resources during
5652    // startup, we assume it came out of startup safemode and it is now in low
5653    // resources safemode
5654    return !safeMode.isManual() && !safeMode.areResourcesLow()
5655      && safeMode.isOn();
5656  }
5657
5658  /**
5659   * Check if replication queues are to be populated
5660   * @return true when node is HAState.Active and not in the very first safemode
5661   */
5662  @Override
5663  public boolean isPopulatingReplQueues() {
5664    if (!shouldPopulateReplQueues()) {
5665      return false;
5666    }
5667    return initializedReplQueues;
5668  }
5669
5670  private boolean shouldPopulateReplQueues() {
5671    if(haContext == null || haContext.getState() == null)
5672      return false;
5673    return haContext.getState().shouldPopulateReplQueues();
5674  }
5675
5676  @Override
5677  public void incrementSafeBlockCount(int replication) {
5678    // safeMode is volatile, and may be set to null at any time
5679    SafeModeInfo safeMode = this.safeMode;
5680    if (safeMode == null)
5681      return;
5682    safeMode.incrementSafeBlockCount((short)replication);
5683  }
5684
5685  @Override
5686  public void decrementSafeBlockCount(Block b) {
5687    // safeMode is volatile, and may be set to null at any time
5688    SafeModeInfo safeMode = this.safeMode;
5689    if (safeMode == null) // mostly true
5690      return;
5691    BlockInfoContiguous storedBlock = getStoredBlock(b);
5692    if (storedBlock.isComplete()) {
5693      safeMode.decrementSafeBlockCount((short)blockManager.countNodes(b).liveReplicas());
5694    }
5695  }
5696  
5697  /**
5698   * Adjust the total number of blocks safe and expected during safe mode.
5699   * If safe mode is not currently on, this is a no-op.
5700   * @param deltaSafe the change in number of safe blocks
5701   * @param deltaTotal the change i nnumber of total blocks expected
5702   */
5703  @Override
5704  public void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal) {
5705    // safeMode is volatile, and may be set to null at any time
5706    SafeModeInfo safeMode = this.safeMode;
5707    if (safeMode == null)
5708      return;
5709    safeMode.adjustBlockTotals(deltaSafe, deltaTotal);
5710  }
5711
5712  /**
5713   * Set the total number of blocks in the system. 
5714   */
5715  public void setBlockTotal() {
5716    // safeMode is volatile, and may be set to null at any time
5717    SafeModeInfo safeMode = this.safeMode;
5718    if (safeMode == null)
5719      return;
5720    safeMode.setBlockTotal((int)getCompleteBlocksTotal());
5721  }
5722
5723  /**
5724   * Get the total number of blocks in the system. 
5725   */
5726  @Override // FSNamesystemMBean
5727  @Metric
5728  public long getBlocksTotal() {
5729    return blockManager.getTotalBlocks();
5730  }
5731
5732  /**
5733   * Get the total number of COMPLETE blocks in the system.
5734   * For safe mode only complete blocks are counted.
5735   */
5736  private long getCompleteBlocksTotal() {
5737    // Calculate number of blocks under construction
5738    long numUCBlocks = 0;
5739    readLock();
5740    numUCBlocks = leaseManager.getNumUnderConstructionBlocks();
5741    try {
5742      return getBlocksTotal() - numUCBlocks;
5743    } finally {
5744      readUnlock();
5745    }
5746  }
5747
5748  /**
5749   * Enter safe mode. If resourcesLow is false, then we assume it is manual
5750   * @throws IOException
5751   */
5752  void enterSafeMode(boolean resourcesLow) throws IOException {
5753    writeLock();
5754    try {
5755      // Stop the secret manager, since rolling the master key would
5756      // try to write to the edit log
5757      stopSecretManager();
5758
5759      // Ensure that any concurrent operations have been fully synced
5760      // before entering safe mode. This ensures that the FSImage
5761      // is entirely stable on disk as soon as we're in safe mode.
5762      boolean isEditlogOpenForWrite = getEditLog().isOpenForWrite();
5763      // Before Editlog is in OpenForWrite mode, editLogStream will be null. So,
5764      // logSyncAll call can be called only when Edlitlog is in OpenForWrite mode
5765      if (isEditlogOpenForWrite) {
5766        getEditLog().logSyncAll();
5767      }
5768      if (!isInSafeMode()) {
5769        safeMode = new SafeModeInfo(resourcesLow);
5770        return;
5771      }
5772      if (resourcesLow) {
5773        safeMode.setResourcesLow();
5774      } else {
5775        safeMode.setManual();
5776      }
5777      if (isEditlogOpenForWrite) {
5778        getEditLog().logSyncAll();
5779      }
5780      NameNode.stateChangeLog.info("STATE* Safe mode is ON"
5781          + safeMode.getTurnOffTip());
5782    } finally {
5783      writeUnlock();
5784    }
5785  }
5786
5787  /**
5788   * Leave safe mode.
5789   */
5790  void leaveSafeMode() {
5791    writeLock();
5792    try {
5793      if (!isInSafeMode()) {
5794        NameNode.stateChangeLog.info("STATE* Safe mode is already OFF"); 
5795        return;
5796      }
5797      safeMode.leave();
5798    } finally {
5799      writeUnlock();
5800    }
5801  }
5802    
5803  String getSafeModeTip() {
5804    // There is no need to take readLock.
5805    // Don't use isInSafeMode as this.safeMode might be set to null.
5806    // after isInSafeMode returns.
5807    boolean inSafeMode;
5808    SafeModeInfo safeMode = this.safeMode;
5809    if (safeMode == null) {
5810      inSafeMode = false;
5811    } else {
5812      inSafeMode = safeMode.isOn();
5813    }
5814
5815    if (!inSafeMode) {
5816      return "";
5817    } else {
5818      return safeMode.getTurnOffTip();
5819    }
5820  }
5821
5822  CheckpointSignature rollEditLog() throws IOException {
5823    checkSuperuserPrivilege();
5824    checkOperation(OperationCategory.JOURNAL);
5825    writeLock();
5826    try {
5827      checkOperation(OperationCategory.JOURNAL);
5828      checkNameNodeSafeMode("Log not rolled");
5829      if (Server.isRpcInvocation()) {
5830        LOG.info("Roll Edit Log from " + Server.getRemoteAddress());
5831      }
5832      return getFSImage().rollEditLog();
5833    } finally {
5834      writeUnlock();
5835    }
5836  }
5837
5838  NamenodeCommand startCheckpoint(NamenodeRegistration backupNode,
5839      NamenodeRegistration activeNamenode) throws IOException {
5840    checkOperation(OperationCategory.CHECKPOINT);
5841    writeLock();
5842    try {
5843      checkOperation(OperationCategory.CHECKPOINT);
5844      checkNameNodeSafeMode("Checkpoint not started");
5845      
5846      LOG.info("Start checkpoint for " + backupNode.getAddress());
5847      NamenodeCommand cmd = getFSImage().startCheckpoint(backupNode,
5848          activeNamenode);
5849      getEditLog().logSync();
5850      return cmd;
5851    } finally {
5852      writeUnlock();
5853    }
5854  }
5855
5856  public void processIncrementalBlockReport(final DatanodeID nodeID,
5857      final StorageReceivedDeletedBlocks srdb)
5858      throws IOException {
5859    writeLock();
5860    try {
5861      blockManager.processIncrementalBlockReport(nodeID, srdb);
5862    } finally {
5863      writeUnlock();
5864    }
5865  }
5866  
5867  void endCheckpoint(NamenodeRegistration registration,
5868                            CheckpointSignature sig) throws IOException {
5869    checkOperation(OperationCategory.CHECKPOINT);
5870    readLock();
5871    try {
5872      checkOperation(OperationCategory.CHECKPOINT);
5873      checkNameNodeSafeMode("Checkpoint not ended");
5874      LOG.info("End checkpoint for " + registration.getAddress());
5875      getFSImage().endCheckpoint(sig);
5876    } finally {
5877      readUnlock();
5878    }
5879  }
5880
5881  PermissionStatus createFsOwnerPermissions(FsPermission permission) {
5882    return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
5883  }
5884
5885  private void checkUnreadableBySuperuser(FSPermissionChecker pc,
5886      INode inode, int snapshotId)
5887      throws IOException {
5888    if (pc.isSuperUser()) {
5889      for (XAttr xattr : FSDirXAttrOp.getXAttrs(dir, inode, snapshotId)) {
5890        if (XAttrHelper.getPrefixName(xattr).
5891            equals(SECURITY_XATTR_UNREADABLE_BY_SUPERUSER)) {
5892          throw new AccessControlException("Access is denied for " +
5893              pc.getUser() + " since the superuser is not allowed to " +
5894              "perform this operation.");
5895        }
5896      }
5897    }
5898  }
5899
5900  @Override
5901  public void checkSuperuserPrivilege()
5902      throws AccessControlException {
5903    if (isPermissionEnabled) {
5904      FSPermissionChecker pc = getPermissionChecker();
5905      pc.checkSuperuserPrivilege();
5906    }
5907  }
5908
5909  /**
5910   * Check to see if we have exceeded the limit on the number
5911   * of inodes.
5912   */
5913  void checkFsObjectLimit() throws IOException {
5914    if (maxFsObjects != 0 &&
5915        maxFsObjects <= dir.totalInodes() + getBlocksTotal()) {
5916      throw new IOException("Exceeded the configured number of objects " +
5917                             maxFsObjects + " in the filesystem.");
5918    }
5919  }
5920
5921  /**
5922   * Get the total number of objects in the system. 
5923   */
5924  @Override // FSNamesystemMBean
5925  public long getMaxObjects() {
5926    return maxFsObjects;
5927  }
5928
5929  @Override // FSNamesystemMBean
5930  @Metric
5931  public long getFilesTotal() {
5932    // There is no need to take fSNamesystem's lock as
5933    // FSDirectory has its own lock.
5934    return this.dir.totalInodes();
5935  }
5936
5937  @Override // FSNamesystemMBean
5938  @Metric
5939  public long getPendingReplicationBlocks() {
5940    return blockManager.getPendingReplicationBlocksCount();
5941  }
5942
5943  @Override // FSNamesystemMBean
5944  @Metric
5945  public long getUnderReplicatedBlocks() {
5946    return blockManager.getUnderReplicatedBlocksCount();
5947  }
5948
5949  /** Returns number of blocks with corrupt replicas */
5950  @Metric({"CorruptBlocks", "Number of blocks with corrupt replicas"})
5951  public long getCorruptReplicaBlocks() {
5952    return blockManager.getCorruptReplicaBlocksCount();
5953  }
5954
5955  @Override // FSNamesystemMBean
5956  @Metric
5957  public long getScheduledReplicationBlocks() {
5958    return blockManager.getScheduledReplicationBlocksCount();
5959  }
5960
5961  @Override
5962  @Metric
5963  public long getPendingDeletionBlocks() {
5964    return blockManager.getPendingDeletionBlocksCount();
5965  }
5966
5967  @Override
5968  public long getBlockDeletionStartTime() {
5969    return startTime + blockManager.getStartupDelayBlockDeletionInMs();
5970  }
5971
5972  @Metric
5973  public long getExcessBlocks() {
5974    return blockManager.getExcessBlocksCount();
5975  }
5976  
5977  // HA-only metric
5978  @Metric
5979  public long getPostponedMisreplicatedBlocks() {
5980    return blockManager.getPostponedMisreplicatedBlocksCount();
5981  }
5982
5983  // HA-only metric
5984  @Metric
5985  public int getPendingDataNodeMessageCount() {
5986    return blockManager.getPendingDataNodeMessageCount();
5987  }
5988  
5989  // HA-only metric
5990  @Metric
5991  public String getHAState() {
5992    return haContext.getState().toString();
5993  }
5994
5995  // HA-only metric
5996  @Metric
5997  public long getMillisSinceLastLoadedEdits() {
5998    if (isInStandbyState() && editLogTailer != null) {
5999      return monotonicNow() - editLogTailer.getLastLoadTimeMs();
6000    } else {
6001      return 0;
6002    }
6003  }
6004  
6005  @Metric
6006  public int getBlockCapacity() {
6007    return blockManager.getCapacity();
6008  }
6009
6010  @Override // FSNamesystemMBean
6011  public String getFSState() {
6012    return isInSafeMode() ? "safeMode" : "Operational";
6013  }
6014  
6015  private ObjectName mbeanName;
6016  private ObjectName mxbeanName;
6017
6018  /**
6019   * Register the FSNamesystem MBean using the name
6020   *        "hadoop:service=NameNode,name=FSNamesystemState"
6021   */
6022  private void registerMBean() {
6023    // We can only implement one MXBean interface, so we keep the old one.
6024    try {
6025      StandardMBean bean = new StandardMBean(this, FSNamesystemMBean.class);
6026      mbeanName = MBeans.register("NameNode", "FSNamesystemState", bean);
6027    } catch (NotCompliantMBeanException e) {
6028      throw new RuntimeException("Bad MBean setup", e);
6029    }
6030
6031    LOG.info("Registered FSNamesystemState MBean");
6032  }
6033
6034  /**
6035   * shutdown FSNamesystem
6036   */
6037  void shutdown() {
6038    if (snapshotManager != null) {
6039      snapshotManager.shutdown();
6040    }
6041    if (mbeanName != null) {
6042      MBeans.unregister(mbeanName);
6043      mbeanName = null;
6044    }
6045    if (mxbeanName != null) {
6046      MBeans.unregister(mxbeanName);
6047      mxbeanName = null;
6048    }
6049    if (dir != null) {
6050      dir.shutdown();
6051    }
6052    if (blockManager != null) {
6053      blockManager.shutdown();
6054    }
6055  }
6056
6057  @Override // FSNamesystemMBean
6058  public int getNumLiveDataNodes() {
6059    return getBlockManager().getDatanodeManager().getNumLiveDataNodes();
6060  }
6061
6062  @Override // FSNamesystemMBean
6063  public int getNumDeadDataNodes() {
6064    return getBlockManager().getDatanodeManager().getNumDeadDataNodes();
6065  }
6066  
6067  @Override // FSNamesystemMBean
6068  public int getNumDecomLiveDataNodes() {
6069    final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
6070    getBlockManager().getDatanodeManager().fetchDatanodes(live, null, true);
6071    int liveDecommissioned = 0;
6072    for (DatanodeDescriptor node : live) {
6073      liveDecommissioned += node.isDecommissioned() ? 1 : 0;
6074    }
6075    return liveDecommissioned;
6076  }
6077
6078  @Override // FSNamesystemMBean
6079  public int getNumDecomDeadDataNodes() {
6080    final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
6081    getBlockManager().getDatanodeManager().fetchDatanodes(null, dead, true);
6082    int deadDecommissioned = 0;
6083    for (DatanodeDescriptor node : dead) {
6084      deadDecommissioned += node.isDecommissioned() ? 1 : 0;
6085    }
6086    return deadDecommissioned;
6087  }
6088
6089  @Override // FSNamesystemMBean
6090  public int getVolumeFailuresTotal() {
6091    List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
6092    getBlockManager().getDatanodeManager().fetchDatanodes(live, null, true);
6093    int volumeFailuresTotal = 0;
6094    for (DatanodeDescriptor node: live) {
6095      volumeFailuresTotal += node.getVolumeFailures();
6096    }
6097    return volumeFailuresTotal;
6098  }
6099
6100  @Override // FSNamesystemMBean
6101  public long getEstimatedCapacityLostTotal() {
6102    List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
6103    getBlockManager().getDatanodeManager().fetchDatanodes(live, null, true);
6104    long estimatedCapacityLostTotal = 0;
6105    for (DatanodeDescriptor node: live) {
6106      VolumeFailureSummary volumeFailureSummary = node.getVolumeFailureSummary();
6107      if (volumeFailureSummary != null) {
6108        estimatedCapacityLostTotal +=
6109            volumeFailureSummary.getEstimatedCapacityLostTotal();
6110      }
6111    }
6112    return estimatedCapacityLostTotal;
6113  }
6114
6115  @Override // FSNamesystemMBean
6116  public int getNumDecommissioningDataNodes() {
6117    return getBlockManager().getDatanodeManager().getDecommissioningNodes()
6118        .size();
6119  }
6120
6121  @Override // FSNamesystemMBean
6122  @Metric({"StaleDataNodes", 
6123    "Number of datanodes marked stale due to delayed heartbeat"})
6124  public int getNumStaleDataNodes() {
6125    return getBlockManager().getDatanodeManager().getNumStaleNodes();
6126  }
6127
6128  /**
6129   * Storages are marked as "content stale" after NN restart or fails over and
6130   * before NN receives the first Heartbeat followed by the first Blockreport.
6131   */
6132  @Override // FSNamesystemMBean
6133  public int getNumStaleStorages() {
6134    return getBlockManager().getDatanodeManager().getNumStaleStorages();
6135  }
6136
6137  @Override // FSNamesystemMBean
6138  public String getTopUserOpCounts() {
6139    if (!topConf.isEnabled) {
6140      return null;
6141    }
6142
6143    Date now = new Date();
6144    final List<RollingWindowManager.TopWindow> topWindows =
6145        topMetrics.getTopWindows();
6146    Map<String, Object> topMap = new TreeMap<String, Object>();
6147    topMap.put("windows", topWindows);
6148    topMap.put("timestamp", DFSUtil.dateToIso8601String(now));
6149    ObjectMapper mapper = new ObjectMapper();
6150    try {
6151      return mapper.writeValueAsString(topMap);
6152    } catch (IOException e) {
6153      LOG.warn("Failed to fetch TopUser metrics", e);
6154    }
6155    return null;
6156  }
6157
6158  /**
6159   * Increments, logs and then returns the stamp
6160   */
6161  long nextGenerationStamp(boolean legacyBlock)
6162      throws IOException, SafeModeException {
6163    assert hasWriteLock();
6164    checkNameNodeSafeMode("Cannot get next generation stamp");
6165
6166    long gs = blockIdManager.nextGenerationStamp(legacyBlock);
6167    if (legacyBlock) {
6168      getEditLog().logGenerationStampV1(gs);
6169    } else {
6170      getEditLog().logGenerationStampV2(gs);
6171    }
6172
6173    // NB: callers sync the log
6174    return gs;
6175  }
6176
6177  /**
6178   * Increments, logs and then returns the block ID
6179   */
6180  private long nextBlockId() throws IOException {
6181    assert hasWriteLock();
6182    checkNameNodeSafeMode("Cannot get next block ID");
6183    final long blockId = blockIdManager.nextBlockId();
6184    getEditLog().logAllocateBlockId(blockId);
6185    // NB: callers sync the log
6186    return blockId;
6187  }
6188
6189  private boolean isFileDeleted(INodeFile file) {
6190    // Not in the inodeMap or in the snapshot but marked deleted.
6191    if (dir.getInode(file.getId()) == null) {
6192      return true;
6193    }
6194
6195    // look at the path hierarchy to see if one parent is deleted by recursive
6196    // deletion
6197    INode tmpChild = file;
6198    INodeDirectory tmpParent = file.getParent();
6199    while (true) {
6200      if (tmpParent == null) {
6201        return true;
6202      }
6203
6204      INode childINode = tmpParent.getChild(tmpChild.getLocalNameBytes(),
6205          Snapshot.CURRENT_STATE_ID);
6206      if (childINode == null || !childINode.equals(tmpChild)) {
6207        // a newly created INode with the same name as an already deleted one
6208        // would be a different INode than the deleted one
6209        return true;
6210      }
6211
6212      if (tmpParent.isRoot()) {
6213        break;
6214      }
6215
6216      tmpChild = tmpParent;
6217      tmpParent = tmpParent.getParent();
6218    }
6219
6220    if (file.isWithSnapshot() &&
6221        file.getFileWithSnapshotFeature().isCurrentFileDeleted()) {
6222      return true;
6223    }
6224    return false;
6225  }
6226
6227  private INodeFile checkUCBlock(ExtendedBlock block,
6228      String clientName) throws IOException {
6229    assert hasWriteLock();
6230    checkNameNodeSafeMode("Cannot get a new generation stamp and an "
6231        + "access token for block " + block);
6232    
6233    // check stored block state
6234    BlockInfoContiguous storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
6235    if (storedBlock == null || 
6236        storedBlock.getBlockUCState() != BlockUCState.UNDER_CONSTRUCTION) {
6237        throw new IOException(block + 
6238            " does not exist or is not under Construction" + storedBlock);
6239    }
6240    
6241    // check file inode
6242    final INodeFile file = ((INode)storedBlock.getBlockCollection()).asFile();
6243    if (file == null || !file.isUnderConstruction() || isFileDeleted(file)) {
6244      throw new IOException("The file " + storedBlock + 
6245          " belonged to does not exist or it is not under construction.");
6246    }
6247    
6248    // check lease
6249    if (clientName == null
6250        || !clientName.equals(file.getFileUnderConstructionFeature()
6251            .getClientName())) {
6252      throw new LeaseExpiredException("Lease mismatch: " + block + 
6253          " is accessed by a non lease holder " + clientName); 
6254    }
6255
6256    return file;
6257  }
6258  
6259  /**
6260   * Client is reporting some bad block locations.
6261   */
6262  void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
6263    checkOperation(OperationCategory.WRITE);
6264    NameNode.stateChangeLog.info("*DIR* reportBadBlocks");
6265    writeLock();
6266    try {
6267      checkOperation(OperationCategory.WRITE);
6268      for (int i = 0; i < blocks.length; i++) {
6269        ExtendedBlock blk = blocks[i].getBlock();
6270        DatanodeInfo[] nodes = blocks[i].getLocations();
6271        String[] storageIDs = blocks[i].getStorageIDs();
6272        for (int j = 0; j < nodes.length; j++) {
6273          blockManager.findAndMarkBlockAsCorrupt(blk, nodes[j],
6274              storageIDs == null ? null: storageIDs[j], 
6275              "client machine reported it");
6276        }
6277      }
6278    } finally {
6279      writeUnlock();
6280    }
6281  }
6282
6283  /**
6284   * Get a new generation stamp together with an access token for 
6285   * a block under construction
6286   * 
6287   * This method is called for recovering a failed pipeline or setting up
6288   * a pipeline to append to a block.
6289   * 
6290   * @param block a block
6291   * @param clientName the name of a client
6292   * @return a located block with a new generation stamp and an access token
6293   * @throws IOException if any error occurs
6294   */
6295  LocatedBlock updateBlockForPipeline(ExtendedBlock block, 
6296      String clientName) throws IOException {
6297    LocatedBlock locatedBlock;
6298    checkOperation(OperationCategory.WRITE);
6299    writeLock();
6300    try {
6301      checkOperation(OperationCategory.WRITE);
6302
6303      // check vadility of parameters
6304      checkUCBlock(block, clientName);
6305  
6306      // get a new generation stamp and an access token
6307      block.setGenerationStamp(nextGenerationStamp(blockIdManager.isLegacyBlock(block.getLocalBlock())));
6308      locatedBlock = new LocatedBlock(block, new DatanodeInfo[0]);
6309      blockManager.setBlockToken(locatedBlock, AccessMode.WRITE);
6310    } finally {
6311      writeUnlock();
6312    }
6313    // Ensure we record the new generation stamp
6314    getEditLog().logSync();
6315    return locatedBlock;
6316  }
6317  
6318  /**
6319   * Update a pipeline for a block under construction
6320   * 
6321   * @param clientName the name of the client
6322   * @param oldBlock and old block
6323   * @param newBlock a new block with a new generation stamp and length
6324   * @param newNodes datanodes in the pipeline
6325   * @throws IOException if any error occurs
6326   */
6327  void updatePipeline(
6328      String clientName, ExtendedBlock oldBlock, ExtendedBlock newBlock,
6329      DatanodeID[] newNodes, String[] newStorageIDs, boolean logRetryCache)
6330      throws IOException {
6331    LOG.info("updatePipeline(" + oldBlock.getLocalBlock()
6332             + ", newGS=" + newBlock.getGenerationStamp()
6333             + ", newLength=" + newBlock.getNumBytes()
6334             + ", newNodes=" + Arrays.asList(newNodes)
6335             + ", client=" + clientName
6336             + ")");
6337    waitForLoadingFSImage();
6338    writeLock();
6339    try {
6340      checkOperation(OperationCategory.WRITE);
6341      checkNameNodeSafeMode("Pipeline not updated");
6342      assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and "
6343        + oldBlock + " has different block identifier";
6344      updatePipelineInternal(clientName, oldBlock, newBlock, newNodes,
6345          newStorageIDs, logRetryCache);
6346    } finally {
6347      writeUnlock();
6348    }
6349    getEditLog().logSync();
6350    LOG.info("updatePipeline(" + oldBlock.getLocalBlock() + " => "
6351        + newBlock.getLocalBlock() + ") success");
6352  }
6353
6354  private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock,
6355      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs,
6356      boolean logRetryCache)
6357      throws IOException {
6358    assert hasWriteLock();
6359    // check the vadility of the block and lease holder name
6360    final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
6361    final BlockInfoContiguousUnderConstruction blockinfo
6362        = (BlockInfoContiguousUnderConstruction)pendingFile.getLastBlock();
6363
6364    // check new GS & length: this is not expected
6365    if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() ||
6366        newBlock.getNumBytes() < blockinfo.getNumBytes()) {
6367      String msg = "Update " + oldBlock + " (len = " + 
6368        blockinfo.getNumBytes() + ") to an older state: " + newBlock + 
6369        " (len = " + newBlock.getNumBytes() +")";
6370      LOG.warn(msg);
6371      throw new IOException(msg);
6372    }
6373
6374    // Update old block with the new generation stamp and new length
6375    blockinfo.setNumBytes(newBlock.getNumBytes());
6376    blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
6377
6378    // find the DatanodeDescriptor objects
6379    final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager()
6380        .getDatanodeStorageInfos(newNodes, newStorageIDs);
6381    blockinfo.setExpectedLocations(storages);
6382
6383    String src = pendingFile.getFullPathName();
6384    persistBlocks(src, pendingFile, logRetryCache);
6385  }
6386
6387  // rename was successful. If any part of the renamed subtree had
6388  // files that were being written to, update with new filename.
6389  void unprotectedChangeLease(String src, String dst) {
6390    assert hasWriteLock();
6391    leaseManager.changeLease(src, dst);
6392  }
6393
6394  /**
6395   * Serializes leases.
6396   */
6397  void saveFilesUnderConstruction(DataOutputStream out,
6398      Map<Long, INodeFile> snapshotUCMap) throws IOException {
6399    // This is run by an inferior thread of saveNamespace, which holds a read
6400    // lock on our behalf. If we took the read lock here, we could block
6401    // for fairness if a writer is waiting on the lock.
6402    synchronized (leaseManager) {
6403      Map<String, INodeFile> nodes = leaseManager.getINodesUnderConstruction();
6404      for (Map.Entry<String, INodeFile> entry : nodes.entrySet()) {
6405        // TODO: for HDFS-5428, because of rename operations, some
6406        // under-construction files that are
6407        // in the current fs directory can also be captured in the
6408        // snapshotUCMap. We should remove them from the snapshotUCMap.
6409        snapshotUCMap.remove(entry.getValue().getId());
6410      }
6411
6412      out.writeInt(nodes.size() + snapshotUCMap.size()); // write the size
6413      for (Map.Entry<String, INodeFile> entry : nodes.entrySet()) {
6414        FSImageSerialization.writeINodeUnderConstruction(
6415            out, entry.getValue(), entry.getKey());
6416      }
6417      for (Map.Entry<Long, INodeFile> entry : snapshotUCMap.entrySet()) {
6418        // for those snapshot INodeFileUC, we use "/.reserved/.inodes/<inodeid>"
6419        // as their paths
6420        StringBuilder b = new StringBuilder();
6421        b.append(FSDirectory.DOT_RESERVED_PATH_PREFIX)
6422            .append(Path.SEPARATOR).append(FSDirectory.DOT_INODES_STRING)
6423            .append(Path.SEPARATOR).append(entry.getValue().getId());
6424        FSImageSerialization.writeINodeUnderConstruction(
6425            out, entry.getValue(), b.toString());
6426      }
6427    }
6428  }
6429
6430  /**
6431   * @return all the under-construction files in the lease map
6432   */
6433  Map<String, INodeFile> getFilesUnderConstruction() {
6434    synchronized (leaseManager) {
6435      return leaseManager.getINodesUnderConstruction();
6436    }
6437  }
6438
6439  /**
6440   * Register a Backup name-node, verifying that it belongs
6441   * to the correct namespace, and adding it to the set of
6442   * active journals if necessary.
6443   * 
6444   * @param bnReg registration of the new BackupNode
6445   * @param nnReg registration of this NameNode
6446   * @throws IOException if the namespace IDs do not match
6447   */
6448  void registerBackupNode(NamenodeRegistration bnReg,
6449      NamenodeRegistration nnReg) throws IOException {
6450    writeLock();
6451    try {
6452      if(getFSImage().getStorage().getNamespaceID() 
6453         != bnReg.getNamespaceID())
6454        throw new IOException("Incompatible namespaceIDs: "
6455            + " Namenode namespaceID = "
6456            + getFSImage().getStorage().getNamespaceID() + "; "
6457            + bnReg.getRole() +
6458            " node namespaceID = " + bnReg.getNamespaceID());
6459      if (bnReg.getRole() == NamenodeRole.BACKUP) {
6460        getFSImage().getEditLog().registerBackupNode(
6461            bnReg, nnReg);
6462      }
6463    } finally {
6464      writeUnlock();
6465    }
6466  }
6467
6468  /**
6469   * Release (unregister) backup node.
6470   * <p>
6471   * Find and remove the backup stream corresponding to the node.
6472   * @throws IOException
6473   */
6474  void releaseBackupNode(NamenodeRegistration registration)
6475    throws IOException {
6476    checkOperation(OperationCategory.WRITE);
6477    writeLock();
6478    try {
6479      checkOperation(OperationCategory.WRITE);
6480      if(getFSImage().getStorage().getNamespaceID()
6481         != registration.getNamespaceID())
6482        throw new IOException("Incompatible namespaceIDs: "
6483            + " Namenode namespaceID = "
6484            + getFSImage().getStorage().getNamespaceID() + "; "
6485            + registration.getRole() +
6486            " node namespaceID = " + registration.getNamespaceID());
6487      getEditLog().releaseBackupStream(registration);
6488    } finally {
6489      writeUnlock();
6490    }
6491  }
6492
6493  static class CorruptFileBlockInfo {
6494    final String path;
6495    final Block block;
6496    
6497    public CorruptFileBlockInfo(String p, Block b) {
6498      path = p;
6499      block = b;
6500    }
6501    
6502    @Override
6503    public String toString() {
6504      return block.getBlockName() + "\t" + path;
6505    }
6506  }
6507  /**
6508   * @param path Restrict corrupt files to this portion of namespace.
6509   * @param cookieTab Support for continuation; cookieTab  tells where
6510   *                  to start from
6511   * @return a list in which each entry describes a corrupt file/block
6512   * @throws IOException
6513   */
6514  Collection<CorruptFileBlockInfo> listCorruptFileBlocks(String path,
6515  String[] cookieTab) throws IOException {
6516    checkSuperuserPrivilege();
6517    checkOperation(OperationCategory.READ);
6518
6519    int count = 0;
6520    ArrayList<CorruptFileBlockInfo> corruptFiles =
6521        new ArrayList<CorruptFileBlockInfo>();
6522    if (cookieTab == null) {
6523      cookieTab = new String[] { null };
6524    }
6525
6526    // Do a quick check if there are any corrupt files without taking the lock
6527    if (blockManager.getMissingBlocksCount() == 0) {
6528      if (cookieTab[0] == null) {
6529        cookieTab[0] = String.valueOf(getIntCookie(cookieTab[0]));
6530      }
6531      if (LOG.isDebugEnabled()) {
6532        LOG.debug("there are no corrupt file blocks.");
6533      }
6534      return corruptFiles;
6535    }
6536
6537    readLock();
6538    try {
6539      checkOperation(OperationCategory.READ);
6540      if (!isPopulatingReplQueues()) {
6541        throw new IOException("Cannot run listCorruptFileBlocks because " +
6542                              "replication queues have not been initialized.");
6543      }
6544      // print a limited # of corrupt files per call
6545
6546      final Iterator<Block> blkIterator = blockManager.getCorruptReplicaBlockIterator();
6547
6548      int skip = getIntCookie(cookieTab[0]);
6549      for (int i = 0; i < skip && blkIterator.hasNext(); i++) {
6550        blkIterator.next();
6551      }
6552
6553      while (blkIterator.hasNext()) {
6554        Block blk = blkIterator.next();
6555        final INode inode = (INode)blockManager.getBlockCollection(blk);
6556        skip++;
6557        if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {
6558          String src = FSDirectory.getFullPathName(inode);
6559          if (src.startsWith(path)){
6560            corruptFiles.add(new CorruptFileBlockInfo(src, blk));
6561            count++;
6562            if (count >= DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED)
6563              break;
6564          }
6565        }
6566      }
6567      cookieTab[0] = String.valueOf(skip);
6568      if (LOG.isDebugEnabled()) {
6569        LOG.debug("list corrupt file blocks returned: " + count);
6570      }
6571      return corruptFiles;
6572    } finally {
6573      readUnlock();
6574    }
6575  }
6576
6577  /**
6578   * Convert string cookie to integer.
6579   */
6580  private static int getIntCookie(String cookie){
6581    int c;
6582    if(cookie == null){
6583      c = 0;
6584    } else {
6585      try{
6586        c = Integer.parseInt(cookie);
6587      }catch (NumberFormatException e) {
6588        c = 0;
6589      }
6590    }
6591    c = Math.max(0, c);
6592    return c;
6593  }
6594
6595  /**
6596   * Create delegation token secret manager
6597   */
6598  private DelegationTokenSecretManager createDelegationTokenSecretManager(
6599      Configuration conf) {
6600    return new DelegationTokenSecretManager(conf.getLong(
6601        DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_KEY,
6602        DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT),
6603        conf.getLong(DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY,
6604            DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT),
6605        conf.getLong(DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY,
6606            DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT),
6607        DELEGATION_TOKEN_REMOVER_SCAN_INTERVAL,
6608        conf.getBoolean(DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_KEY,
6609            DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_DEFAULT),
6610        this);
6611  }
6612
6613  /**
6614   * Returns the DelegationTokenSecretManager instance in the namesystem.
6615   * @return delegation token secret manager object
6616   */
6617  DelegationTokenSecretManager getDelegationTokenSecretManager() {
6618    return dtSecretManager;
6619  }
6620
6621  /**
6622   * @param renewer Renewer information
6623   * @return delegation toek
6624   * @throws IOException on error
6625   */
6626  Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
6627      throws IOException {
6628    Token<DelegationTokenIdentifier> token;
6629    checkOperation(OperationCategory.WRITE);
6630    writeLock();
6631    try {
6632      checkOperation(OperationCategory.WRITE);
6633      checkNameNodeSafeMode("Cannot issue delegation token");
6634      if (!isAllowedDelegationTokenOp()) {
6635        throw new IOException(
6636          "Delegation Token can be issued only with kerberos or web authentication");
6637      }
6638      if (dtSecretManager == null || !dtSecretManager.isRunning()) {
6639        LOG.warn("trying to get DT with no secret manager running");
6640        return null;
6641      }
6642
6643      UserGroupInformation ugi = getRemoteUser();
6644      String user = ugi.getUserName();
6645      Text owner = new Text(user);
6646      Text realUser = null;
6647      if (ugi.getRealUser() != null) {
6648        realUser = new Text(ugi.getRealUser().getUserName());
6649      }
6650      DelegationTokenIdentifier dtId = new DelegationTokenIdentifier(owner,
6651        renewer, realUser);
6652      token = new Token<DelegationTokenIdentifier>(
6653        dtId, dtSecretManager);
6654      long expiryTime = dtSecretManager.getTokenExpiryTime(dtId);
6655      getEditLog().logGetDelegationToken(dtId, expiryTime);
6656    } finally {
6657      writeUnlock();
6658    }
6659    getEditLog().logSync();
6660    return token;
6661  }
6662
6663  /**
6664   * 
6665   * @param token token to renew
6666   * @return new expiryTime of the token
6667   * @throws InvalidToken if {@code token} is invalid
6668   * @throws IOException on other errors
6669   */
6670  long renewDelegationToken(Token<DelegationTokenIdentifier> token)
6671      throws InvalidToken, IOException {
6672    long expiryTime;
6673    checkOperation(OperationCategory.WRITE);
6674    writeLock();
6675    try {
6676      checkOperation(OperationCategory.WRITE);
6677
6678      checkNameNodeSafeMode("Cannot renew delegation token");
6679      if (!isAllowedDelegationTokenOp()) {
6680        throw new IOException(
6681            "Delegation Token can be renewed only with kerberos or web authentication");
6682      }
6683      String renewer = getRemoteUser().getShortUserName();
6684      expiryTime = dtSecretManager.renewToken(token, renewer);
6685      DelegationTokenIdentifier id = new DelegationTokenIdentifier();
6686      ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
6687      DataInputStream in = new DataInputStream(buf);
6688      id.readFields(in);
6689      getEditLog().logRenewDelegationToken(id, expiryTime);
6690    } finally {
6691      writeUnlock();
6692    }
6693    getEditLog().logSync();
6694    return expiryTime;
6695  }
6696
6697  /**
6698   * 
6699   * @param token token to cancel
6700   * @throws IOException on error
6701   */
6702  void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
6703      throws IOException {
6704    checkOperation(OperationCategory.WRITE);
6705    writeLock();
6706    try {
6707      checkOperation(OperationCategory.WRITE);
6708
6709      checkNameNodeSafeMode("Cannot cancel delegation token");
6710      String canceller = getRemoteUser().getUserName();
6711      DelegationTokenIdentifier id = dtSecretManager
6712        .cancelToken(token, canceller);
6713      getEditLog().logCancelDelegationToken(id);
6714    } finally {
6715      writeUnlock();
6716    }
6717    getEditLog().logSync();
6718  }
6719
6720  /**
6721   * @param out save state of the secret manager
6722   * @param sdPath String storage directory path
6723   */
6724  void saveSecretManagerStateCompat(DataOutputStream out, String sdPath)
6725      throws IOException {
6726    dtSecretManager.saveSecretManagerStateCompat(out, sdPath);
6727  }
6728
6729  SecretManagerState saveSecretManagerState() {
6730    return dtSecretManager.saveSecretManagerState();
6731  }
6732
6733  /**
6734   * @param in load the state of secret manager from input stream
6735   */
6736  void loadSecretManagerStateCompat(DataInput in) throws IOException {
6737    dtSecretManager.loadSecretManagerStateCompat(in);
6738  }
6739
6740  void loadSecretManagerState(SecretManagerSection s,
6741      List<SecretManagerSection.DelegationKey> keys,
6742      List<SecretManagerSection.PersistToken> tokens) throws IOException {
6743    dtSecretManager.loadSecretManagerState(new SecretManagerState(s, keys, tokens));
6744  }
6745
6746  /**
6747   * Log the updateMasterKey operation to edit logs
6748   * 
6749   * @param key new delegation key.
6750   */
6751  public void logUpdateMasterKey(DelegationKey key) {
6752    
6753    assert !isInSafeMode() :
6754      "this should never be called while in safemode, since we stop " +
6755      "the DT manager before entering safemode!";
6756    // No need to hold FSN lock since we don't access any internal
6757    // structures, and this is stopped before the FSN shuts itself
6758    // down, etc.
6759    getEditLog().logUpdateMasterKey(key);
6760    getEditLog().logSync();
6761  }
6762  
6763  /**
6764   * Log the cancellation of expired tokens to edit logs
6765   * 
6766   * @param id token identifier to cancel
6767   */
6768  public void logExpireDelegationToken(DelegationTokenIdentifier id) {
6769    assert !isInSafeMode() :
6770      "this should never be called while in safemode, since we stop " +
6771      "the DT manager before entering safemode!";
6772    // No need to hold FSN lock since we don't access any internal
6773    // structures, and this is stopped before the FSN shuts itself
6774    // down, etc.
6775    getEditLog().logCancelDelegationToken(id);
6776  }  
6777  
6778  private void logReassignLease(String leaseHolder, String src,
6779      String newHolder) {
6780    assert hasWriteLock();
6781    getEditLog().logReassignLease(leaseHolder, src, newHolder);
6782  }
6783  
6784  /**
6785   * 
6786   * @return true if delegation token operation is allowed
6787   */
6788  private boolean isAllowedDelegationTokenOp() throws IOException {
6789    AuthenticationMethod authMethod = getConnectionAuthenticationMethod();
6790    if (UserGroupInformation.isSecurityEnabled()
6791        && (authMethod != AuthenticationMethod.KERBEROS)
6792        && (authMethod != AuthenticationMethod.KERBEROS_SSL)
6793        && (authMethod != AuthenticationMethod.CERTIFICATE)) {
6794      return false;
6795    }
6796    return true;
6797  }
6798  
6799  /**
6800   * Returns authentication method used to establish the connection
6801   * @return AuthenticationMethod used to establish connection
6802   * @throws IOException
6803   */
6804  private AuthenticationMethod getConnectionAuthenticationMethod()
6805      throws IOException {
6806    UserGroupInformation ugi = getRemoteUser();
6807    AuthenticationMethod authMethod = ugi.getAuthenticationMethod();
6808    if (authMethod == AuthenticationMethod.PROXY) {
6809      authMethod = ugi.getRealUser().getAuthenticationMethod();
6810    }
6811    return authMethod;
6812  }
6813  
6814  /**
6815   * Client invoked methods are invoked over RPC and will be in 
6816   * RPC call context even if the client exits.
6817   */
6818  boolean isExternalInvocation() {
6819    return Server.isRpcInvocation() || NamenodeWebHdfsMethods.isWebHdfsInvocation();
6820  }
6821
6822  private static InetAddress getRemoteIp() {
6823    InetAddress ip = Server.getRemoteIp();
6824    if (ip != null) {
6825      return ip;
6826    }
6827    return NamenodeWebHdfsMethods.getRemoteIp();
6828  }
6829  
6830  // optimize ugi lookup for RPC operations to avoid a trip through
6831  // UGI.getCurrentUser which is synch'ed
6832  private static UserGroupInformation getRemoteUser() throws IOException {
6833    return NameNode.getRemoteUser();
6834  }
6835  
6836  /**
6837   * Log fsck event in the audit log 
6838   */
6839  void logFsckEvent(String src, InetAddress remoteAddress) throws IOException {
6840    if (isAuditEnabled()) {
6841      logAuditEvent(true, getRemoteUser(),
6842                    remoteAddress,
6843                    "fsck", src, null, null);
6844    }
6845  }
6846  /**
6847   * Register NameNodeMXBean
6848   */
6849  private void registerMXBean() {
6850    mxbeanName = MBeans.register("NameNode", "NameNodeInfo", this);
6851  }
6852
6853  /**
6854   * Class representing Namenode information for JMX interfaces
6855   */
6856  @Override // NameNodeMXBean
6857  public String getVersion() {
6858    return VersionInfo.getVersion() + ", r" + VersionInfo.getRevision();
6859  }
6860
6861  @Override // NameNodeMXBean
6862  public long getUsed() {
6863    return this.getCapacityUsed();
6864  }
6865
6866  @Override // NameNodeMXBean
6867  public long getFree() {
6868    return this.getCapacityRemaining();
6869  }
6870
6871  @Override // NameNodeMXBean
6872  public long getTotal() {
6873    return this.getCapacityTotal();
6874  }
6875
6876  @Override // NameNodeMXBean
6877  public String getSafemode() {
6878    if (!this.isInSafeMode())
6879      return "";
6880    return "Safe mode is ON. " + this.getSafeModeTip();
6881  }
6882
6883  @Override // NameNodeMXBean
6884  public boolean isUpgradeFinalized() {
6885    return this.getFSImage().isUpgradeFinalized();
6886  }
6887
6888  @Override // NameNodeMXBean
6889  public long getNonDfsUsedSpace() {
6890    return datanodeStatistics.getCapacityUsedNonDFS();
6891  }
6892
6893  @Override // NameNodeMXBean
6894  public float getPercentUsed() {
6895    return datanodeStatistics.getCapacityUsedPercent();
6896  }
6897
6898  @Override // NameNodeMXBean
6899  public long getBlockPoolUsedSpace() {
6900    return datanodeStatistics.getBlockPoolUsed();
6901  }
6902
6903  @Override // NameNodeMXBean
6904  public float getPercentBlockPoolUsed() {
6905    return datanodeStatistics.getPercentBlockPoolUsed();
6906  }
6907
6908  @Override // NameNodeMXBean
6909  public float getPercentRemaining() {
6910    return datanodeStatistics.getCapacityRemainingPercent();
6911  }
6912
6913  @Override // NameNodeMXBean
6914  public long getCacheCapacity() {
6915    return datanodeStatistics.getCacheCapacity();
6916  }
6917
6918  @Override // NameNodeMXBean
6919  public long getCacheUsed() {
6920    return datanodeStatistics.getCacheUsed();
6921  }
6922
6923  @Override // NameNodeMXBean
6924  public long getTotalBlocks() {
6925    return getBlocksTotal();
6926  }
6927
6928  @Override // NameNodeMXBean
6929  @Metric
6930  public long getTotalFiles() {
6931    return getFilesTotal();
6932  }
6933
6934  @Override // NameNodeMXBean
6935  public long getNumberOfMissingBlocks() {
6936    return getMissingBlocksCount();
6937  }
6938  
6939  @Override // NameNodeMXBean
6940  public long getNumberOfMissingBlocksWithReplicationFactorOne() {
6941    return getMissingReplOneBlocksCount();
6942  }
6943
6944  @Override // NameNodeMXBean
6945  public int getThreads() {
6946    return ManagementFactory.getThreadMXBean().getThreadCount();
6947  }
6948
6949  /**
6950   * Returned information is a JSON representation of map with host name as the
6951   * key and value is a map of live node attribute keys to its values
6952   */
6953  @Override // NameNodeMXBean
6954  public String getLiveNodes() {
6955    final Map<String, Map<String,Object>> info = 
6956      new HashMap<String, Map<String,Object>>();
6957    final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
6958    blockManager.getDatanodeManager().fetchDatanodes(live, null, true);
6959    for (DatanodeDescriptor node : live) {
6960      ImmutableMap.Builder<String, Object> innerinfo =
6961          ImmutableMap.<String,Object>builder();
6962      innerinfo
6963          .put("infoAddr", node.getInfoAddr())
6964          .put("infoSecureAddr", node.getInfoSecureAddr())
6965          .put("xferaddr", node.getXferAddr())
6966          .put("lastContact", getLastContact(node))
6967          .put("usedSpace", getDfsUsed(node))
6968          .put("adminState", node.getAdminState().toString())
6969          .put("nonDfsUsedSpace", node.getNonDfsUsed())
6970          .put("capacity", node.getCapacity())
6971          .put("numBlocks", node.numBlocks())
6972          .put("version", node.getSoftwareVersion())
6973          .put("used", node.getDfsUsed())
6974          .put("remaining", node.getRemaining())
6975          .put("blockScheduled", node.getBlocksScheduled())
6976          .put("blockPoolUsed", node.getBlockPoolUsed())
6977          .put("blockPoolUsedPercent", node.getBlockPoolUsedPercent())
6978          .put("volfails", node.getVolumeFailures());
6979      VolumeFailureSummary volumeFailureSummary = node.getVolumeFailureSummary();
6980      if (volumeFailureSummary != null) {
6981        innerinfo
6982            .put("failedStorageLocations",
6983                volumeFailureSummary.getFailedStorageLocations())
6984            .put("lastVolumeFailureDate",
6985                volumeFailureSummary.getLastVolumeFailureDate())
6986            .put("estimatedCapacityLostTotal",
6987                volumeFailureSummary.getEstimatedCapacityLostTotal());
6988      }
6989      info.put(node.getHostName() + ":" + node.getXferPort(), innerinfo.build());
6990    }
6991    return JSON.toString(info);
6992  }
6993
6994  /**
6995   * Returned information is a JSON representation of map with host name as the
6996   * key and value is a map of dead node attribute keys to its values
6997   */
6998  @Override // NameNodeMXBean
6999  public String getDeadNodes() {
7000    final Map<String, Map<String, Object>> info = 
7001      new HashMap<String, Map<String, Object>>();
7002    final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
7003    blockManager.getDatanodeManager().fetchDatanodes(null, dead, true);
7004    for (DatanodeDescriptor node : dead) {
7005      Map<String, Object> innerinfo = ImmutableMap.<String, Object>builder()
7006          .put("lastContact", getLastContact(node))
7007          .put("decommissioned", node.isDecommissioned())
7008          .put("xferaddr", node.getXferAddr())
7009          .build();
7010      info.put(node.getHostName() + ":" + node.getXferPort(), innerinfo);
7011    }
7012    return JSON.toString(info);
7013  }
7014
7015  /**
7016   * Returned information is a JSON representation of map with host name as the
7017   * key and value is a map of decommissioning node attribute keys to its
7018   * values
7019   */
7020  @Override // NameNodeMXBean
7021  public String getDecomNodes() {
7022    final Map<String, Map<String, Object>> info = 
7023      new HashMap<String, Map<String, Object>>();
7024    final List<DatanodeDescriptor> decomNodeList = blockManager.getDatanodeManager(
7025        ).getDecommissioningNodes();
7026    for (DatanodeDescriptor node : decomNodeList) {
7027      Map<String, Object> innerinfo = ImmutableMap
7028          .<String, Object> builder()
7029          .put("xferaddr", node.getXferAddr())
7030          .put("underReplicatedBlocks",
7031              node.decommissioningStatus.getUnderReplicatedBlocks())
7032          .put("decommissionOnlyReplicas",
7033              node.decommissioningStatus.getDecommissionOnlyReplicas())
7034          .put("underReplicateInOpenFiles",
7035              node.decommissioningStatus.getUnderReplicatedInOpenFiles())
7036          .build();
7037      info.put(node.getHostName() + ":" + node.getXferPort(), innerinfo);
7038    }
7039    return JSON.toString(info);
7040  }
7041
7042  private long getLastContact(DatanodeDescriptor alivenode) {
7043    return (monotonicNow() - alivenode.getLastUpdateMonotonic())/1000;
7044  }
7045
7046  private long getDfsUsed(DatanodeDescriptor alivenode) {
7047    return alivenode.getDfsUsed();
7048  }
7049
7050  @Override  // NameNodeMXBean
7051  public String getClusterId() {
7052    return getFSImage().getStorage().getClusterID();
7053  }
7054  
7055  @Override  // NameNodeMXBean
7056  public String getBlockPoolId() {
7057    return blockPoolId;
7058  }
7059  
7060  @Override  // NameNodeMXBean
7061  public String getNameDirStatuses() {
7062    Map<String, Map<File, StorageDirType>> statusMap =
7063      new HashMap<String, Map<File, StorageDirType>>();
7064    
7065    Map<File, StorageDirType> activeDirs = new HashMap<File, StorageDirType>();
7066    for (Iterator<StorageDirectory> it
7067        = getFSImage().getStorage().dirIterator(); it.hasNext();) {
7068      StorageDirectory st = it.next();
7069      activeDirs.put(st.getRoot(), st.getStorageDirType());
7070    }
7071    statusMap.put("active", activeDirs);
7072    
7073    List<Storage.StorageDirectory> removedStorageDirs
7074        = getFSImage().getStorage().getRemovedStorageDirs();
7075    Map<File, StorageDirType> failedDirs = new HashMap<File, StorageDirType>();
7076    for (StorageDirectory st : removedStorageDirs) {
7077      failedDirs.put(st.getRoot(), st.getStorageDirType());
7078    }
7079    statusMap.put("failed", failedDirs);
7080    
7081    return JSON.toString(statusMap);
7082  }
7083
7084  @Override // NameNodeMXBean
7085  public String getNodeUsage() {
7086    float median = 0;
7087    float max = 0;
7088    float min = 0;
7089    float dev = 0;
7090
7091    final Map<String, Map<String,Object>> info =
7092        new HashMap<String, Map<String,Object>>();
7093    final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
7094    blockManager.getDatanodeManager().fetchDatanodes(live, null, true);
7095
7096    if (live.size() > 0) {
7097      float totalDfsUsed = 0;
7098      float[] usages = new float[live.size()];
7099      int i = 0;
7100      for (DatanodeDescriptor dn : live) {
7101        usages[i++] = dn.getDfsUsedPercent();
7102        totalDfsUsed += dn.getDfsUsedPercent();
7103      }
7104      totalDfsUsed /= live.size();
7105      Arrays.sort(usages);
7106      median = usages[usages.length / 2];
7107      max = usages[usages.length - 1];
7108      min = usages[0];
7109
7110      for (i = 0; i < usages.length; i++) {
7111        dev += (usages[i] - totalDfsUsed) * (usages[i] - totalDfsUsed);
7112      }
7113      dev = (float) Math.sqrt(dev / usages.length);
7114    }
7115
7116    final Map<String, Object> innerInfo = new HashMap<String, Object>();
7117    innerInfo.put("min", StringUtils.format("%.2f%%", min));
7118    innerInfo.put("median", StringUtils.format("%.2f%%", median));
7119    innerInfo.put("max", StringUtils.format("%.2f%%", max));
7120    innerInfo.put("stdDev", StringUtils.format("%.2f%%", dev));
7121    info.put("nodeUsage", innerInfo);
7122
7123    return JSON.toString(info);
7124  }
7125
7126  @Override  // NameNodeMXBean
7127  public String getNameJournalStatus() {
7128    List<Map<String, String>> jasList = new ArrayList<Map<String, String>>();
7129    FSEditLog log = getFSImage().getEditLog();
7130    if (log != null) {
7131      boolean openForWrite = log.isOpenForWrite();
7132      for (JournalAndStream jas : log.getJournals()) {
7133        final Map<String, String> jasMap = new HashMap<String, String>();
7134        String manager = jas.getManager().toString();
7135
7136        jasMap.put("required", String.valueOf(jas.isRequired()));
7137        jasMap.put("disabled", String.valueOf(jas.isDisabled()));
7138        jasMap.put("manager", manager);
7139
7140        if (jas.isDisabled()) {
7141          jasMap.put("stream", "Failed");
7142        } else if (openForWrite) {
7143          EditLogOutputStream elos = jas.getCurrentStream();
7144          if (elos != null) {
7145            jasMap.put("stream", elos.generateReport());
7146          } else {
7147            jasMap.put("stream", "not currently writing");
7148          }
7149        } else {
7150          jasMap.put("stream", "open for read");
7151        }
7152        jasList.add(jasMap);
7153      }
7154    }
7155    return JSON.toString(jasList);
7156  }
7157
7158  @Override // NameNodeMxBean
7159  public String getJournalTransactionInfo() {
7160    Map<String, String> txnIdMap = new HashMap<String, String>();
7161    txnIdMap.put("LastAppliedOrWrittenTxId",
7162        Long.toString(this.getFSImage().getLastAppliedOrWrittenTxId()));
7163    txnIdMap.put("MostRecentCheckpointTxId",
7164        Long.toString(this.getFSImage().getMostRecentCheckpointTxId()));
7165    return JSON.toString(txnIdMap);
7166  }
7167  
7168  @Override  // NameNodeMXBean
7169  public String getNNStarted() {
7170    return getStartTime().toString();
7171  }
7172
7173  @Override  // NameNodeMXBean
7174  public String getCompileInfo() {
7175    return VersionInfo.getDate() + " by " + VersionInfo.getUser() +
7176        " from " + VersionInfo.getBranch();
7177  }
7178
7179  /** @return the block manager. */
7180  public BlockManager getBlockManager() {
7181    return blockManager;
7182  }
7183
7184  public BlockIdManager getBlockIdManager() {
7185    return blockIdManager;
7186  }
7187
7188  /** @return the FSDirectory. */
7189  public FSDirectory getFSDirectory() {
7190    return dir;
7191  }
7192  /** Set the FSDirectory. */
7193  @VisibleForTesting
7194  public void setFSDirectory(FSDirectory dir) {
7195    this.dir = dir;
7196  }
7197  /** @return the cache manager. */
7198  public CacheManager getCacheManager() {
7199    return cacheManager;
7200  }
7201
7202  @Override  // NameNodeMXBean
7203  public String getCorruptFiles() {
7204    List<String> list = new ArrayList<String>();
7205    Collection<FSNamesystem.CorruptFileBlockInfo> corruptFileBlocks;
7206    try {
7207      corruptFileBlocks = listCorruptFileBlocks("/", null);
7208      int corruptFileCount = corruptFileBlocks.size();
7209      if (corruptFileCount != 0) {
7210        for (FSNamesystem.CorruptFileBlockInfo c : corruptFileBlocks) {
7211          list.add(c.toString());
7212        }
7213      }
7214    } catch (IOException e) {
7215      LOG.warn("Get corrupt file blocks returned error: " + e.getMessage());
7216    }
7217    return JSON.toString(list);
7218  }
7219
7220  @Override  //NameNodeMXBean
7221  public int getDistinctVersionCount() {
7222    return blockManager.getDatanodeManager().getDatanodesSoftwareVersions()
7223      .size();
7224  }
7225
7226  @Override  //NameNodeMXBean
7227  public Map<String, Integer> getDistinctVersions() {
7228    return blockManager.getDatanodeManager().getDatanodesSoftwareVersions();
7229  }
7230
7231  @Override  //NameNodeMXBean
7232  public String getSoftwareVersion() {
7233    return VersionInfo.getVersion();
7234  }
7235
7236  /**
7237   * Verifies that the given identifier and password are valid and match.
7238   * @param identifier Token identifier.
7239   * @param password Password in the token.
7240   */
7241  public synchronized void verifyToken(DelegationTokenIdentifier identifier,
7242      byte[] password) throws InvalidToken, RetriableException {
7243    try {
7244      getDelegationTokenSecretManager().verifyToken(identifier, password);
7245    } catch (InvalidToken it) {
7246      if (inTransitionToActive()) {
7247        throw new RetriableException(it);
7248      }
7249      throw it;
7250    }
7251  }
7252  
7253  @Override
7254  public boolean isGenStampInFuture(Block block) {
7255    return blockIdManager.isGenStampInFuture(block);
7256  }
7257
7258  @VisibleForTesting
7259  public EditLogTailer getEditLogTailer() {
7260    return editLogTailer;
7261  }
7262  
7263  @VisibleForTesting
7264  public void setEditLogTailerForTests(EditLogTailer tailer) {
7265    this.editLogTailer = tailer;
7266  }
7267  
7268  @VisibleForTesting
7269  void setFsLockForTests(ReentrantReadWriteLock lock) {
7270    this.fsLock.coarseLock = lock;
7271  }
7272  
7273  @VisibleForTesting
7274  public ReentrantReadWriteLock getFsLockForTests() {
7275    return fsLock.coarseLock;
7276  }
7277  
7278  @VisibleForTesting
7279  public ReentrantLock getCpLockForTests() {
7280    return cpLock;
7281  }
7282
7283  @VisibleForTesting
7284  public SafeModeInfo getSafeModeInfoForTests() {
7285    return safeMode;
7286  }
7287  
7288  @VisibleForTesting
7289  public void setNNResourceChecker(NameNodeResourceChecker nnResourceChecker) {
7290    this.nnResourceChecker = nnResourceChecker;
7291  }
7292
7293  public SnapshotManager getSnapshotManager() {
7294    return snapshotManager;
7295  }
7296  
7297  /** Allow snapshot on a directory. */
7298  void allowSnapshot(String path) throws IOException {
7299    checkOperation(OperationCategory.WRITE);
7300    boolean success = false;
7301    writeLock();
7302    try {
7303      checkOperation(OperationCategory.WRITE);
7304      checkNameNodeSafeMode("Cannot allow snapshot for " + path);
7305      checkSuperuserPrivilege();
7306      FSDirSnapshotOp.allowSnapshot(dir, snapshotManager, path);
7307      success = true;
7308    } finally {
7309      writeUnlock();
7310    }
7311    getEditLog().logSync();
7312    logAuditEvent(success, "allowSnapshot", path, null, null);
7313  }
7314  
7315  /** Disallow snapshot on a directory. */
7316  void disallowSnapshot(String path) throws IOException {
7317    checkOperation(OperationCategory.WRITE);
7318    boolean success = false;
7319    writeLock();
7320    try {
7321      checkOperation(OperationCategory.WRITE);
7322      checkNameNodeSafeMode("Cannot disallow snapshot for " + path);
7323      checkSuperuserPrivilege();
7324      FSDirSnapshotOp.disallowSnapshot(dir, snapshotManager, path);
7325      success = true;
7326    } finally {
7327      writeUnlock();
7328    }
7329    getEditLog().logSync();
7330    logAuditEvent(success, "disallowSnapshot", path, null, null);
7331  }
7332  
7333  /**
7334   * Create a snapshot
7335   * @param snapshotRoot The directory path where the snapshot is taken
7336   * @param snapshotName The name of the snapshot
7337   */
7338  String createSnapshot(String snapshotRoot, String snapshotName,
7339                        boolean logRetryCache) throws IOException {
7340    String snapshotPath = null;
7341    writeLock();
7342    try {
7343      checkOperation(OperationCategory.WRITE);
7344      checkNameNodeSafeMode("Cannot create snapshot for " + snapshotRoot);
7345      snapshotPath = FSDirSnapshotOp.createSnapshot(dir,
7346          snapshotManager, snapshotRoot, snapshotName, logRetryCache);
7347    } finally {
7348      writeUnlock();
7349    }
7350    getEditLog().logSync();
7351    logAuditEvent(snapshotPath != null, "createSnapshot", snapshotRoot,
7352        snapshotPath, null);
7353    return snapshotPath;
7354  }
7355  
7356  /**
7357   * Rename a snapshot
7358   * @param path The directory path where the snapshot was taken
7359   * @param snapshotOldName Old snapshot name
7360   * @param snapshotNewName New snapshot name
7361   * @throws SafeModeException
7362   * @throws IOException 
7363   */
7364  void renameSnapshot(
7365      String path, String snapshotOldName, String snapshotNewName,
7366      boolean logRetryCache) throws IOException {
7367    boolean success = false;
7368    writeLock();
7369    try {
7370      checkOperation(OperationCategory.WRITE);
7371      checkNameNodeSafeMode("Cannot rename snapshot for " + path);
7372      FSDirSnapshotOp.renameSnapshot(dir, snapshotManager, path,
7373          snapshotOldName, snapshotNewName, logRetryCache);
7374      success = true;
7375    } finally {
7376      writeUnlock();
7377    }
7378    getEditLog().logSync();
7379    String oldSnapshotRoot = Snapshot.getSnapshotPath(path, snapshotOldName);
7380    String newSnapshotRoot = Snapshot.getSnapshotPath(path, snapshotNewName);
7381    logAuditEvent(success, "renameSnapshot", oldSnapshotRoot,
7382        newSnapshotRoot, null);
7383  }
7384  
7385  /**
7386   * Get the list of snapshottable directories that are owned 
7387   * by the current user. Return all the snapshottable directories if the 
7388   * current user is a super user.
7389   * @return The list of all the current snapshottable directories
7390   * @throws IOException
7391   */
7392  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
7393      throws IOException {
7394    SnapshottableDirectoryStatus[] status = null;
7395    checkOperation(OperationCategory.READ);
7396    boolean success = false;
7397    readLock();
7398    try {
7399      checkOperation(OperationCategory.READ);
7400      status = FSDirSnapshotOp.getSnapshottableDirListing(dir, snapshotManager);
7401      success = true;
7402    } finally {
7403      readUnlock();
7404    }
7405    logAuditEvent(success, "listSnapshottableDirectory", null, null, null);
7406    return status;
7407  }
7408  
7409  /**
7410   * Get the difference between two snapshots (or between a snapshot and the
7411   * current status) of a snapshottable directory.
7412   * 
7413   * @param path The full path of the snapshottable directory.
7414   * @param fromSnapshot Name of the snapshot to calculate the diff from. Null
7415   *          or empty string indicates the current tree.
7416   * @param toSnapshot Name of the snapshot to calculated the diff to. Null or
7417   *          empty string indicates the current tree.
7418   * @return A report about the difference between {@code fromSnapshot} and 
7419   *         {@code toSnapshot}. Modified/deleted/created/renamed files and 
7420   *         directories belonging to the snapshottable directories are listed 
7421   *         and labeled as M/-/+/R respectively. 
7422   * @throws IOException
7423   */
7424  SnapshotDiffReport getSnapshotDiffReport(String path,
7425      String fromSnapshot, String toSnapshot) throws IOException {
7426    SnapshotDiffReport diffs = null;
7427    checkOperation(OperationCategory.READ);
7428    readLock();
7429    try {
7430      checkOperation(OperationCategory.READ);
7431      diffs = FSDirSnapshotOp.getSnapshotDiffReport(dir, snapshotManager,
7432          path, fromSnapshot, toSnapshot);
7433    } finally {
7434      readUnlock();
7435    }
7436
7437    logAuditEvent(diffs != null, "computeSnapshotDiff", null, null, null);
7438    return diffs;
7439  }
7440  
7441  /**
7442   * Delete a snapshot of a snapshottable directory
7443   * @param snapshotRoot The snapshottable directory
7444   * @param snapshotName The name of the to-be-deleted snapshot
7445   * @throws SafeModeException
7446   * @throws IOException
7447   */
7448  void deleteSnapshot(String snapshotRoot, String snapshotName,
7449      boolean logRetryCache) throws IOException {
7450    boolean success = false;
7451    writeLock();
7452    BlocksMapUpdateInfo blocksToBeDeleted = null;
7453    try {
7454      checkOperation(OperationCategory.WRITE);
7455      checkNameNodeSafeMode("Cannot delete snapshot for " + snapshotRoot);
7456
7457      blocksToBeDeleted = FSDirSnapshotOp.deleteSnapshot(dir, snapshotManager,
7458          snapshotRoot, snapshotName, logRetryCache);
7459      success = true;
7460    } finally {
7461      writeUnlock();
7462    }
7463    getEditLog().logSync();
7464
7465    // Breaking the pattern as removing blocks have to happen outside of the
7466    // global lock
7467    if (blocksToBeDeleted != null) {
7468      removeBlocks(blocksToBeDeleted);
7469    }
7470
7471    String rootPath = Snapshot.getSnapshotPath(snapshotRoot, snapshotName);
7472    logAuditEvent(success, "deleteSnapshot", rootPath, null, null);
7473  }
7474
7475  /**
7476   * Remove a list of INodeDirectorySnapshottable from the SnapshotManager
7477   * @param toRemove the list of INodeDirectorySnapshottable to be removed
7478   */
7479  void removeSnapshottableDirs(List<INodeDirectory> toRemove) {
7480    if (snapshotManager != null) {
7481      snapshotManager.removeSnapshottable(toRemove);
7482    }
7483  }
7484
7485  RollingUpgradeInfo queryRollingUpgrade() throws IOException {
7486    checkSuperuserPrivilege();
7487    checkOperation(OperationCategory.READ);
7488    readLock();
7489    try {
7490      if (!isRollingUpgrade()) {
7491        return null;
7492      }
7493      Preconditions.checkNotNull(rollingUpgradeInfo);
7494      boolean hasRollbackImage = this.getFSImage().hasRollbackFSImage();
7495      rollingUpgradeInfo.setCreatedRollbackImages(hasRollbackImage);
7496      return rollingUpgradeInfo;
7497    } finally {
7498      readUnlock();
7499    }
7500  }
7501
7502  RollingUpgradeInfo startRollingUpgrade() throws IOException {
7503    checkSuperuserPrivilege();
7504    checkOperation(OperationCategory.WRITE);
7505    writeLock();
7506    try {
7507      checkOperation(OperationCategory.WRITE);
7508      if (isRollingUpgrade()) {
7509        return rollingUpgradeInfo;
7510      }
7511      long startTime = now();
7512      if (!haEnabled) { // for non-HA, we require NN to be in safemode
7513        startRollingUpgradeInternalForNonHA(startTime);
7514      } else { // for HA, NN cannot be in safemode
7515        checkNameNodeSafeMode("Failed to start rolling upgrade");
7516        startRollingUpgradeInternal(startTime);
7517      }
7518
7519      getEditLog().logStartRollingUpgrade(rollingUpgradeInfo.getStartTime());
7520      if (haEnabled) {
7521        // roll the edit log to make sure the standby NameNode can tail
7522        getFSImage().rollEditLog();
7523      }
7524    } finally {
7525      writeUnlock();
7526    }
7527
7528    getEditLog().logSync();
7529    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
7530      logAuditEvent(true, "startRollingUpgrade", null, null, null);
7531    }
7532    return rollingUpgradeInfo;
7533  }
7534
7535  /**
7536   * Update internal state to indicate that a rolling upgrade is in progress.
7537   * @param startTime rolling upgrade start time
7538   */
7539  void startRollingUpgradeInternal(long startTime)
7540      throws IOException {
7541    checkRollingUpgrade("start rolling upgrade");
7542    getFSImage().checkUpgrade();
7543    setRollingUpgradeInfo(false, startTime);
7544  }
7545
7546  /**
7547   * Update internal state to indicate that a rolling upgrade is in progress for
7548   * non-HA setup. This requires the namesystem is in SafeMode and after doing a
7549   * checkpoint for rollback the namesystem will quit the safemode automatically 
7550   */
7551  private void startRollingUpgradeInternalForNonHA(long startTime)
7552      throws IOException {
7553    Preconditions.checkState(!haEnabled);
7554    if (!isInSafeMode()) {
7555      throw new IOException("Safe mode should be turned ON "
7556          + "in order to create namespace image.");
7557    }
7558    checkRollingUpgrade("start rolling upgrade");
7559    getFSImage().checkUpgrade();
7560    // in non-HA setup, we do an extra checkpoint to generate a rollback image
7561    getFSImage().saveNamespace(this, NameNodeFile.IMAGE_ROLLBACK, null);
7562    LOG.info("Successfully saved namespace for preparing rolling upgrade.");
7563
7564    // leave SafeMode automatically
7565    setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
7566    setRollingUpgradeInfo(true, startTime);
7567  }
7568
7569  void setRollingUpgradeInfo(boolean createdRollbackImages, long startTime) {
7570    rollingUpgradeInfo = new RollingUpgradeInfo(blockPoolId,
7571        createdRollbackImages, startTime, 0L);
7572  }
7573
7574  public void setCreatedRollbackImages(boolean created) {
7575    if (rollingUpgradeInfo != null) {
7576      rollingUpgradeInfo.setCreatedRollbackImages(created);
7577    }
7578  }
7579
7580  public RollingUpgradeInfo getRollingUpgradeInfo() {
7581    return rollingUpgradeInfo;
7582  }
7583
7584  public boolean isNeedRollbackFsImage() {
7585    return needRollbackFsImage;
7586  }
7587
7588  public void setNeedRollbackFsImage(boolean needRollbackFsImage) {
7589    this.needRollbackFsImage = needRollbackFsImage;
7590  }
7591
7592  @Override  // NameNodeMXBean
7593  public RollingUpgradeInfo.Bean getRollingUpgradeStatus() {
7594    if (!isRollingUpgrade()) {
7595      return null;
7596    }
7597    RollingUpgradeInfo upgradeInfo = getRollingUpgradeInfo();
7598    if (upgradeInfo.createdRollbackImages()) {
7599      return new RollingUpgradeInfo.Bean(upgradeInfo);
7600    }
7601    readLock();
7602    try {
7603      // check again after acquiring the read lock.
7604      upgradeInfo = getRollingUpgradeInfo();
7605      if (upgradeInfo == null) {
7606        return null;
7607      }
7608      if (!upgradeInfo.createdRollbackImages()) {
7609        boolean hasRollbackImage = this.getFSImage().hasRollbackFSImage();
7610        upgradeInfo.setCreatedRollbackImages(hasRollbackImage);
7611      }
7612    } catch (IOException ioe) {
7613      LOG.warn("Encountered exception setting Rollback Image", ioe);
7614    } finally {
7615      readUnlock();
7616    }
7617    return new RollingUpgradeInfo.Bean(upgradeInfo);
7618  }
7619
7620  /** Is rolling upgrade in progress? */
7621  public boolean isRollingUpgrade() {
7622    return rollingUpgradeInfo != null && !rollingUpgradeInfo.isFinalized();
7623  }
7624
7625  void checkRollingUpgrade(String action) throws RollingUpgradeException {
7626    if (isRollingUpgrade()) {
7627      throw new RollingUpgradeException("Failed to " + action
7628          + " since a rolling upgrade is already in progress."
7629          + " Existing rolling upgrade info:\n" + rollingUpgradeInfo);
7630    }
7631  }
7632
7633  RollingUpgradeInfo finalizeRollingUpgrade() throws IOException {
7634    checkSuperuserPrivilege();
7635    checkOperation(OperationCategory.WRITE);
7636    writeLock();
7637    try {
7638      checkOperation(OperationCategory.WRITE);
7639      if (!isRollingUpgrade()) {
7640        return null;
7641      }
7642      checkNameNodeSafeMode("Failed to finalize rolling upgrade");
7643
7644      finalizeRollingUpgradeInternal(now());
7645      getEditLog().logFinalizeRollingUpgrade(rollingUpgradeInfo.getFinalizeTime());
7646      if (haEnabled) {
7647        // roll the edit log to make sure the standby NameNode can tail
7648        getFSImage().rollEditLog();
7649      }
7650      getFSImage().updateStorageVersion();
7651      getFSImage().renameCheckpoint(NameNodeFile.IMAGE_ROLLBACK,
7652          NameNodeFile.IMAGE);
7653    } finally {
7654      writeUnlock();
7655    }
7656
7657    if (!haEnabled) {
7658      // Sync not needed for ha since the edit was rolled after logging.
7659      getEditLog().logSync();
7660    }
7661
7662    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
7663      logAuditEvent(true, "finalizeRollingUpgrade", null, null, null);
7664    }
7665    return rollingUpgradeInfo;
7666  }
7667
7668  void finalizeRollingUpgradeInternal(long finalizeTime) {
7669    // Set the finalize time
7670    rollingUpgradeInfo.finalize(finalizeTime);
7671  }
7672
7673  long addCacheDirective(CacheDirectiveInfo directive,
7674                         EnumSet<CacheFlag> flags, boolean logRetryCache)
7675      throws IOException {
7676    CacheDirectiveInfo effectiveDirective = null;
7677    if (!flags.contains(CacheFlag.FORCE)) {
7678      cacheManager.waitForRescanIfNeeded();
7679    }
7680    writeLock();
7681    try {
7682      checkOperation(OperationCategory.WRITE);
7683      if (isInSafeMode()) {
7684        throw new SafeModeException(
7685            "Cannot add cache directive", safeMode);
7686      }
7687      effectiveDirective = FSNDNCacheOp.addCacheDirective(this, cacheManager,
7688          directive, flags, logRetryCache);
7689    } finally {
7690      writeUnlock();
7691      boolean success = effectiveDirective != null;
7692      if (success) {
7693        getEditLog().logSync();
7694      }
7695
7696      String effectiveDirectiveStr = effectiveDirective != null ?
7697          effectiveDirective.toString() : null;
7698      logAuditEvent(success, "addCacheDirective", effectiveDirectiveStr,
7699          null, null);
7700    }
7701    return effectiveDirective != null ? effectiveDirective.getId() : 0;
7702  }
7703
7704  void modifyCacheDirective(CacheDirectiveInfo directive,
7705      EnumSet<CacheFlag> flags, boolean logRetryCache) throws IOException {
7706    boolean success = false;
7707    if (!flags.contains(CacheFlag.FORCE)) {
7708      cacheManager.waitForRescanIfNeeded();
7709    }
7710    writeLock();
7711    try {
7712      checkOperation(OperationCategory.WRITE);
7713      if (isInSafeMode()) {
7714        throw new SafeModeException(
7715            "Cannot add cache directive", safeMode);
7716      }
7717      FSNDNCacheOp.modifyCacheDirective(this, cacheManager, directive, flags,
7718          logRetryCache);
7719      success = true;
7720    } finally {
7721      writeUnlock();
7722      if (success) {
7723        getEditLog().logSync();
7724      }
7725      String idStr = "{id: " + directive.getId().toString() + "}";
7726      logAuditEvent(success, "modifyCacheDirective", idStr,
7727          directive.toString(), null);
7728    }
7729  }
7730
7731  void removeCacheDirective(long id, boolean logRetryCache) throws IOException {
7732    boolean success = false;
7733    writeLock();
7734    try {
7735      checkOperation(OperationCategory.WRITE);
7736      if (isInSafeMode()) {
7737        throw new SafeModeException(
7738            "Cannot remove cache directives", safeMode);
7739      }
7740      FSNDNCacheOp.removeCacheDirective(this, cacheManager, id, logRetryCache);
7741      success = true;
7742    } finally {
7743      writeUnlock();
7744      String idStr = "{id: " + Long.toString(id) + "}";
7745      logAuditEvent(success, "removeCacheDirective", idStr, null,
7746          null);
7747    }
7748    getEditLog().logSync();
7749  }
7750
7751  BatchedListEntries<CacheDirectiveEntry> listCacheDirectives(
7752      long startId, CacheDirectiveInfo filter) throws IOException {
7753    checkOperation(OperationCategory.READ);
7754    BatchedListEntries<CacheDirectiveEntry> results;
7755    cacheManager.waitForRescanIfNeeded();
7756    readLock();
7757    boolean success = false;
7758    try {
7759      checkOperation(OperationCategory.READ);
7760      results = FSNDNCacheOp.listCacheDirectives(this, cacheManager, startId,
7761          filter);
7762      success = true;
7763    } finally {
7764      readUnlock();
7765      logAuditEvent(success, "listCacheDirectives", filter.toString(), null,
7766          null);
7767    }
7768    return results;
7769  }
7770
7771  void addCachePool(CachePoolInfo req, boolean logRetryCache)
7772      throws IOException {
7773    writeLock();
7774    boolean success = false;
7775    String poolInfoStr = null;
7776    try {
7777      checkOperation(OperationCategory.WRITE);
7778      if (isInSafeMode()) {
7779        throw new SafeModeException(
7780            "Cannot add cache pool " + req.getPoolName(), safeMode);
7781      }
7782      CachePoolInfo info = FSNDNCacheOp.addCachePool(this, cacheManager, req,
7783          logRetryCache);
7784      poolInfoStr = info.toString();
7785      success = true;
7786    } finally {
7787      writeUnlock();
7788      logAuditEvent(success, "addCachePool", poolInfoStr, null, null);
7789    }
7790    
7791    getEditLog().logSync();
7792  }
7793
7794  void modifyCachePool(CachePoolInfo req, boolean logRetryCache)
7795      throws IOException {
7796    writeLock();
7797    boolean success = false;
7798    try {
7799      checkOperation(OperationCategory.WRITE);
7800      if (isInSafeMode()) {
7801        throw new SafeModeException(
7802            "Cannot modify cache pool " + req.getPoolName(), safeMode);
7803      }
7804      FSNDNCacheOp.modifyCachePool(this, cacheManager, req, logRetryCache);
7805      success = true;
7806    } finally {
7807      writeUnlock();
7808      String poolNameStr = "{poolName: " +
7809          (req == null ? null : req.getPoolName()) + "}";
7810      logAuditEvent(success, "modifyCachePool", poolNameStr,
7811                    req == null ? null : req.toString(), null);
7812    }
7813
7814    getEditLog().logSync();
7815  }
7816
7817  void removeCachePool(String cachePoolName, boolean logRetryCache)
7818      throws IOException {
7819    writeLock();
7820    boolean success = false;
7821    try {
7822      checkOperation(OperationCategory.WRITE);
7823      if (isInSafeMode()) {
7824        throw new SafeModeException(
7825            "Cannot remove cache pool " + cachePoolName, safeMode);
7826      }
7827      FSNDNCacheOp.removeCachePool(this, cacheManager, cachePoolName,
7828          logRetryCache);
7829      success = true;
7830    } finally {
7831      writeUnlock();
7832      String poolNameStr = "{poolName: " + cachePoolName + "}";
7833      logAuditEvent(success, "removeCachePool", poolNameStr, null, null);
7834    }
7835    
7836    getEditLog().logSync();
7837  }
7838
7839  BatchedListEntries<CachePoolEntry> listCachePools(String prevKey)
7840      throws IOException {
7841    BatchedListEntries<CachePoolEntry> results;
7842    checkOperation(OperationCategory.READ);
7843    boolean success = false;
7844    cacheManager.waitForRescanIfNeeded();
7845    readLock();
7846    try {
7847      checkOperation(OperationCategory.READ);
7848      results = FSNDNCacheOp.listCachePools(this, cacheManager, prevKey);
7849      success = true;
7850    } finally {
7851      readUnlock();
7852      logAuditEvent(success, "listCachePools", null, null, null);
7853    }
7854    return results;
7855  }
7856
7857  void modifyAclEntries(final String src, List<AclEntry> aclSpec)
7858      throws IOException {
7859    HdfsFileStatus auditStat = null;
7860    checkOperation(OperationCategory.WRITE);
7861    writeLock();
7862    try {
7863      checkOperation(OperationCategory.WRITE);
7864      checkNameNodeSafeMode("Cannot modify ACL entries on " + src);
7865      auditStat = FSDirAclOp.modifyAclEntries(dir, src, aclSpec);
7866    } catch (AccessControlException e) {
7867      logAuditEvent(false, "modifyAclEntries", src);
7868      throw e;
7869    } finally {
7870      writeUnlock();
7871    }
7872    getEditLog().logSync();
7873    logAuditEvent(true, "modifyAclEntries", src, null, auditStat);
7874  }
7875
7876  void removeAclEntries(final String src, List<AclEntry> aclSpec)
7877      throws IOException {
7878    checkOperation(OperationCategory.WRITE);
7879    HdfsFileStatus auditStat = null;
7880    writeLock();
7881    try {
7882      checkOperation(OperationCategory.WRITE);
7883      checkNameNodeSafeMode("Cannot remove ACL entries on " + src);
7884      auditStat = FSDirAclOp.removeAclEntries(dir, src, aclSpec);
7885    } catch (AccessControlException e) {
7886      logAuditEvent(false, "removeAclEntries", src);
7887      throw e;
7888    } finally {
7889      writeUnlock();
7890    }
7891    getEditLog().logSync();
7892    logAuditEvent(true, "removeAclEntries", src, null, auditStat);
7893  }
7894
7895  void removeDefaultAcl(final String src) throws IOException {
7896    HdfsFileStatus auditStat = null;
7897    checkOperation(OperationCategory.WRITE);
7898    writeLock();
7899    try {
7900      checkOperation(OperationCategory.WRITE);
7901      checkNameNodeSafeMode("Cannot remove default ACL entries on " + src);
7902      auditStat = FSDirAclOp.removeDefaultAcl(dir, src);
7903    } catch (AccessControlException e) {
7904      logAuditEvent(false, "removeDefaultAcl", src);
7905      throw e;
7906    } finally {
7907      writeUnlock();
7908    }
7909    getEditLog().logSync();
7910    logAuditEvent(true, "removeDefaultAcl", src, null, auditStat);
7911  }
7912
7913  void removeAcl(final String src) throws IOException {
7914    HdfsFileStatus auditStat = null;
7915    checkOperation(OperationCategory.WRITE);
7916    writeLock();
7917    try {
7918      checkOperation(OperationCategory.WRITE);
7919      checkNameNodeSafeMode("Cannot remove ACL on " + src);
7920      auditStat = FSDirAclOp.removeAcl(dir, src);
7921    } catch (AccessControlException e) {
7922      logAuditEvent(false, "removeAcl", src);
7923      throw e;
7924    } finally {
7925      writeUnlock();
7926    }
7927    getEditLog().logSync();
7928    logAuditEvent(true, "removeAcl", src, null, auditStat);
7929  }
7930
7931  void setAcl(final String src, List<AclEntry> aclSpec) throws IOException {
7932    HdfsFileStatus auditStat = null;
7933    checkOperation(OperationCategory.WRITE);
7934    writeLock();
7935    try {
7936      checkOperation(OperationCategory.WRITE);
7937      checkNameNodeSafeMode("Cannot set ACL on " + src);
7938      auditStat = FSDirAclOp.setAcl(dir, src, aclSpec);
7939    } catch (AccessControlException e) {
7940      logAuditEvent(false, "setAcl", src);
7941      throw e;
7942    } finally {
7943      writeUnlock();
7944    }
7945    getEditLog().logSync();
7946    logAuditEvent(true, "setAcl", src, null, auditStat);
7947  }
7948
7949  AclStatus getAclStatus(String src) throws IOException {
7950    checkOperation(OperationCategory.READ);
7951    boolean success = false;
7952    readLock();
7953    try {
7954      checkOperation(OperationCategory.READ);
7955      final AclStatus ret = FSDirAclOp.getAclStatus(dir, src);
7956      success = true;
7957      return ret;
7958    } finally {
7959      readUnlock();
7960      logAuditEvent(success, "getAclStatus", src);
7961    }
7962  }
7963
7964  /**
7965   * Create an encryption zone on directory src using the specified key.
7966   *
7967   * @param src     the path of a directory which will be the root of the
7968   *                encryption zone. The directory must be empty.
7969   * @param keyName name of a key which must be present in the configured
7970   *                KeyProvider.
7971   * @throws AccessControlException  if the caller is not the superuser.
7972   * @throws UnresolvedLinkException if the path can't be resolved.
7973   * @throws SafeModeException       if the Namenode is in safe mode.
7974   */
7975  void createEncryptionZone(final String src, final String keyName,
7976                            boolean logRetryCache)
7977    throws IOException, UnresolvedLinkException,
7978      SafeModeException, AccessControlException {
7979    try {
7980      if (provider == null) {
7981        throw new IOException(
7982            "Can't create an encryption zone for " + src +
7983            " since no key provider is available.");
7984      }
7985      if (keyName == null || keyName.isEmpty()) {
7986        throw new IOException("Must specify a key name when creating an " +
7987            "encryption zone");
7988      }
7989      KeyProvider.Metadata metadata = provider.getMetadata(keyName);
7990      if (metadata == null) {
7991        /*
7992         * It would be nice if we threw something more specific than
7993         * IOException when the key is not found, but the KeyProvider API
7994         * doesn't provide for that. If that API is ever changed to throw
7995         * something more specific (e.g. UnknownKeyException) then we can
7996         * update this to match it, or better yet, just rethrow the
7997         * KeyProvider's exception.
7998         */
7999        throw new IOException("Key " + keyName + " doesn't exist.");
8000      }
8001      // If the provider supports pool for EDEKs, this will fill in the pool
8002      generateEncryptedDataEncryptionKey(keyName);
8003      createEncryptionZoneInt(src, metadata.getCipher(),
8004          keyName, logRetryCache);
8005    } catch (AccessControlException e) {
8006      logAuditEvent(false, "createEncryptionZone", src);
8007      throw e;
8008    }
8009  }
8010
8011  private void createEncryptionZoneInt(final String srcArg, String cipher,
8012      String keyName, final boolean logRetryCache) throws IOException {
8013    String src = srcArg;
8014    HdfsFileStatus resultingStat = null;
8015    checkSuperuserPrivilege();
8016    final byte[][] pathComponents =
8017      FSDirectory.getPathComponentsForReservedPath(src);
8018    FSPermissionChecker pc = getPermissionChecker();
8019    writeLock();
8020    try {
8021      checkSuperuserPrivilege();
8022      checkOperation(OperationCategory.WRITE);
8023      checkNameNodeSafeMode("Cannot create encryption zone on " + src);
8024      src = dir.resolvePath(pc, src, pathComponents);
8025
8026      final CipherSuite suite = CipherSuite.convert(cipher);
8027      // For now this is hardcoded, as we only support one method.
8028      final CryptoProtocolVersion version =
8029          CryptoProtocolVersion.ENCRYPTION_ZONES;
8030      final XAttr ezXAttr = dir.createEncryptionZone(src, suite,
8031          version, keyName);
8032      List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
8033      xAttrs.add(ezXAttr);
8034      getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
8035      final INodesInPath iip = dir.getINodesInPath4Write(src, false);
8036      resultingStat = dir.getAuditFileInfo(iip);
8037    } finally {
8038      writeUnlock();
8039    }
8040    getEditLog().logSync();
8041    logAuditEvent(true, "createEncryptionZone", srcArg, null, resultingStat);
8042  }
8043
8044  /**
8045   * Get the encryption zone for the specified path.
8046   *
8047   * @param srcArg the path of a file or directory to get the EZ for.
8048   * @return the EZ of the of the path or null if none.
8049   * @throws AccessControlException  if the caller is not the superuser.
8050   * @throws UnresolvedLinkException if the path can't be resolved.
8051   */
8052  EncryptionZone getEZForPath(final String srcArg)
8053    throws AccessControlException, UnresolvedLinkException, IOException {
8054    String src = srcArg;
8055    HdfsFileStatus resultingStat = null;
8056    final byte[][] pathComponents =
8057        FSDirectory.getPathComponentsForReservedPath(src);
8058    boolean success = false;
8059    final FSPermissionChecker pc = getPermissionChecker();
8060    checkOperation(OperationCategory.READ);
8061    readLock();
8062    try {
8063      checkOperation(OperationCategory.READ);
8064      src = dir.resolvePath(pc, src, pathComponents);
8065      final INodesInPath iip = dir.getINodesInPath(src, true);
8066      if (isPermissionEnabled) {
8067        dir.checkPathAccess(pc, iip, FsAction.READ);
8068      }
8069      final EncryptionZone ret = dir.getEZForPath(iip);
8070      resultingStat = dir.getAuditFileInfo(iip);
8071      success = true;
8072      return ret;
8073    } finally {
8074      readUnlock();
8075      logAuditEvent(success, "getEZForPath", srcArg, null, resultingStat);
8076    }
8077  }
8078
8079  BatchedListEntries<EncryptionZone> listEncryptionZones(long prevId)
8080      throws IOException {
8081    boolean success = false;
8082    checkSuperuserPrivilege();
8083    checkOperation(OperationCategory.READ);
8084    readLock();
8085    try {
8086      checkSuperuserPrivilege();
8087      checkOperation(OperationCategory.READ);
8088      final BatchedListEntries<EncryptionZone> ret =
8089          dir.listEncryptionZones(prevId);
8090      success = true;
8091      return ret;
8092    } finally {
8093      readUnlock();
8094      logAuditEvent(success, "listEncryptionZones", null);
8095    }
8096  }
8097
8098  void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag,
8099                boolean logRetryCache)
8100      throws IOException {
8101    HdfsFileStatus auditStat = null;
8102    writeLock();
8103    try {
8104      checkOperation(OperationCategory.WRITE);
8105      checkNameNodeSafeMode("Cannot set XAttr on " + src);
8106      auditStat = FSDirXAttrOp.setXAttr(dir, src, xAttr, flag, logRetryCache);
8107    } catch (AccessControlException e) {
8108      logAuditEvent(false, "setXAttr", src);
8109      throw e;
8110    } finally {
8111      writeUnlock();
8112    }
8113    getEditLog().logSync();
8114    logAuditEvent(true, "setXAttr", src, null, auditStat);
8115  }
8116
8117  List<XAttr> getXAttrs(final String src, List<XAttr> xAttrs)
8118      throws IOException {
8119    checkOperation(OperationCategory.READ);
8120    readLock();
8121    try {
8122      checkOperation(OperationCategory.READ);
8123      return FSDirXAttrOp.getXAttrs(dir, src, xAttrs);
8124    } catch (AccessControlException e) {
8125      logAuditEvent(false, "getXAttrs", src);
8126      throw e;
8127    } finally {
8128      readUnlock();
8129    }
8130  }
8131
8132  List<XAttr> listXAttrs(String src) throws IOException {
8133    checkOperation(OperationCategory.READ);
8134    readLock();
8135    try {
8136      checkOperation(OperationCategory.READ);
8137      return FSDirXAttrOp.listXAttrs(dir, src);
8138    } catch (AccessControlException e) {
8139      logAuditEvent(false, "listXAttrs", src);
8140      throw e;
8141    } finally {
8142      readUnlock();
8143    }
8144  }
8145
8146  void removeXAttr(String src, XAttr xAttr, boolean logRetryCache)
8147      throws IOException {
8148    HdfsFileStatus auditStat = null;
8149    writeLock();
8150    try {
8151      checkOperation(OperationCategory.WRITE);
8152      checkNameNodeSafeMode("Cannot remove XAttr entry on " + src);
8153      auditStat = FSDirXAttrOp.removeXAttr(dir, src, xAttr, logRetryCache);
8154    } catch (AccessControlException e) {
8155      logAuditEvent(false, "removeXAttr", src);
8156      throw e;
8157    } finally {
8158      writeUnlock();
8159    }
8160    getEditLog().logSync();
8161    logAuditEvent(true, "removeXAttr", src, null, auditStat);
8162  }
8163
8164  void checkAccess(String src, FsAction mode) throws IOException {
8165    checkOperation(OperationCategory.READ);
8166    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
8167    readLock();
8168    try {
8169      checkOperation(OperationCategory.READ);
8170      src = FSDirectory.resolvePath(src, pathComponents, dir);
8171      final INodesInPath iip = dir.getINodesInPath(src, true);
8172      INode inode = iip.getLastINode();
8173      if (inode == null) {
8174        throw new FileNotFoundException("Path not found");
8175      }
8176      if (isPermissionEnabled) {
8177        FSPermissionChecker pc = getPermissionChecker();
8178        dir.checkPathAccess(pc, iip, mode);
8179      }
8180    } catch (AccessControlException e) {
8181      logAuditEvent(false, "checkAccess", src);
8182      throw e;
8183    } finally {
8184      readUnlock();
8185    }
8186  }
8187
8188  /**
8189   * Default AuditLogger implementation; used when no access logger is
8190   * defined in the config file. It can also be explicitly listed in the
8191   * config file.
8192   */
8193  private static class DefaultAuditLogger extends HdfsAuditLogger {
8194
8195    private boolean logTokenTrackingId;
8196
8197    @Override
8198    public void initialize(Configuration conf) {
8199      logTokenTrackingId = conf.getBoolean(
8200          DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_KEY,
8201          DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_DEFAULT);
8202    }
8203
8204    @Override
8205    public void logAuditEvent(boolean succeeded, String userName,
8206        InetAddress addr, String cmd, String src, String dst,
8207        FileStatus status, UserGroupInformation ugi,
8208        DelegationTokenSecretManager dtSecretManager) {
8209      if (auditLog.isInfoEnabled()) {
8210        final StringBuilder sb = auditBuffer.get();
8211        sb.setLength(0);
8212        sb.append("allowed=").append(succeeded).append("\t");
8213        sb.append("ugi=").append(userName).append("\t");
8214        sb.append("ip=").append(addr).append("\t");
8215        sb.append("cmd=").append(cmd).append("\t");
8216        sb.append("src=").append(src).append("\t");
8217        sb.append("dst=").append(dst).append("\t");
8218        if (null == status) {
8219          sb.append("perm=null");
8220        } else {
8221          sb.append("perm=");
8222          sb.append(status.getOwner()).append(":");
8223          sb.append(status.getGroup()).append(":");
8224          sb.append(status.getPermission());
8225        }
8226        if (logTokenTrackingId) {
8227          sb.append("\t").append("trackingId=");
8228          String trackingId = null;
8229          if (ugi != null && dtSecretManager != null
8230              && ugi.getAuthenticationMethod() == AuthenticationMethod.TOKEN) {
8231            for (TokenIdentifier tid: ugi.getTokenIdentifiers()) {
8232              if (tid instanceof DelegationTokenIdentifier) {
8233                DelegationTokenIdentifier dtid =
8234                    (DelegationTokenIdentifier)tid;
8235                trackingId = dtSecretManager.getTokenTrackingId(dtid);
8236                break;
8237              }
8238            }
8239          }
8240          sb.append(trackingId);
8241        }
8242        sb.append("\t").append("proto=");
8243        sb.append(NamenodeWebHdfsMethods.isWebHdfsInvocation() ? "webhdfs" : "rpc");
8244        logAuditMessage(sb.toString());
8245      }
8246    }
8247
8248    public void logAuditMessage(String message) {
8249      auditLog.info(message);
8250    }
8251  }
8252
8253  private static void enableAsyncAuditLog() {
8254    if (!(auditLog instanceof Log4JLogger)) {
8255      LOG.warn("Log4j is required to enable async auditlog");
8256      return;
8257    }
8258    Logger logger = ((Log4JLogger)auditLog).getLogger();
8259    @SuppressWarnings("unchecked")
8260    List<Appender> appenders = Collections.list(logger.getAllAppenders());
8261    // failsafe against trying to async it more than once
8262    if (!appenders.isEmpty() && !(appenders.get(0) instanceof AsyncAppender)) {
8263      AsyncAppender asyncAppender = new AsyncAppender();
8264      // change logger to have an async appender containing all the
8265      // previously configured appenders
8266      for (Appender appender : appenders) {
8267        logger.removeAppender(appender);
8268        asyncAppender.addAppender(appender);
8269      }
8270      logger.addAppender(asyncAppender);        
8271    }
8272  }
8273
8274}
8275