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