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 java.io.PrintStream;
021import java.io.PrintWriter;
022import java.io.StringWriter;
023import java.util.List;
024
025import org.apache.commons.logging.Log;
026import org.apache.commons.logging.LogFactory;
027import org.apache.hadoop.classification.InterfaceAudience;
028import org.apache.hadoop.fs.ContentSummary;
029import org.apache.hadoop.fs.Path;
030import org.apache.hadoop.fs.permission.FsPermission;
031import org.apache.hadoop.fs.permission.PermissionStatus;
032import org.apache.hadoop.hdfs.protocol.HdfsConstants;
033import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
034import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
035import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
036import org.apache.hadoop.hdfs.DFSUtil;
037import org.apache.hadoop.hdfs.protocol.Block;
038import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
039import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
040import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
041import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
042import org.apache.hadoop.hdfs.util.Diff;
043import org.apache.hadoop.util.ChunkedArrayList;
044import org.apache.hadoop.util.StringUtils;
045
046import com.google.common.annotations.VisibleForTesting;
047import com.google.common.base.Preconditions;
048
049/**
050 * We keep an in-memory representation of the file/block hierarchy.
051 * This is a base INode class containing common fields for file and 
052 * directory inodes.
053 */
054@InterfaceAudience.Private
055public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
056  public static final Log LOG = LogFactory.getLog(INode.class);
057
058  /** parent is either an {@link INodeDirectory} or an {@link INodeReference}.*/
059  private INode parent = null;
060
061  INode(INode parent) {
062    this.parent = parent;
063  }
064
065  /** Get inode id */
066  public abstract long getId();
067
068  /**
069   * Check whether this is the root inode.
070   */
071  final boolean isRoot() {
072    return getLocalNameBytes().length == 0;
073  }
074
075  /** Get the {@link PermissionStatus} */
076  abstract PermissionStatus getPermissionStatus(int snapshotId);
077
078  /** The same as getPermissionStatus(null). */
079  final PermissionStatus getPermissionStatus() {
080    return getPermissionStatus(Snapshot.CURRENT_STATE_ID);
081  }
082
083  /**
084   * @param snapshotId
085   *          if it is not {@link Snapshot#CURRENT_STATE_ID}, get the result
086   *          from the given snapshot; otherwise, get the result from the
087   *          current inode.
088   * @return user name
089   */
090  abstract String getUserName(int snapshotId);
091
092  /** The same as getUserName(Snapshot.CURRENT_STATE_ID). */
093  @Override
094  public final String getUserName() {
095    return getUserName(Snapshot.CURRENT_STATE_ID);
096  }
097
098  /** Set user */
099  abstract void setUser(String user);
100
101  /** Set user */
102  final INode setUser(String user, int latestSnapshotId) {
103    recordModification(latestSnapshotId);
104    setUser(user);
105    return this;
106  }
107  /**
108   * @param snapshotId
109   *          if it is not {@link Snapshot#CURRENT_STATE_ID}, get the result
110   *          from the given snapshot; otherwise, get the result from the
111   *          current inode.
112   * @return group name
113   */
114  abstract String getGroupName(int snapshotId);
115
116  /** The same as getGroupName(Snapshot.CURRENT_STATE_ID). */
117  @Override
118  public final String getGroupName() {
119    return getGroupName(Snapshot.CURRENT_STATE_ID);
120  }
121
122  /** Set group */
123  abstract void setGroup(String group);
124
125  /** Set group */
126  final INode setGroup(String group, int latestSnapshotId) {
127    recordModification(latestSnapshotId);
128    setGroup(group);
129    return this;
130  }
131
132  /**
133   * @param snapshotId
134   *          if it is not {@link Snapshot#CURRENT_STATE_ID}, get the result
135   *          from the given snapshot; otherwise, get the result from the
136   *          current inode.
137   * @return permission.
138   */
139  abstract FsPermission getFsPermission(int snapshotId);
140  
141  /** The same as getFsPermission(Snapshot.CURRENT_STATE_ID). */
142  @Override
143  public final FsPermission getFsPermission() {
144    return getFsPermission(Snapshot.CURRENT_STATE_ID);
145  }
146
147  /** Set the {@link FsPermission} of this {@link INode} */
148  abstract void setPermission(FsPermission permission);
149
150  /** Set the {@link FsPermission} of this {@link INode} */
151  INode setPermission(FsPermission permission, int latestSnapshotId) {
152    recordModification(latestSnapshotId);
153    setPermission(permission);
154    return this;
155  }
156
157  abstract AclFeature getAclFeature(int snapshotId);
158
159  @Override
160  public final AclFeature getAclFeature() {
161    return getAclFeature(Snapshot.CURRENT_STATE_ID);
162  }
163
164  abstract void addAclFeature(AclFeature aclFeature);
165
166  final INode addAclFeature(AclFeature aclFeature, int latestSnapshotId) {
167    recordModification(latestSnapshotId);
168    addAclFeature(aclFeature);
169    return this;
170  }
171
172  abstract void removeAclFeature();
173
174  final INode removeAclFeature(int latestSnapshotId) {
175    recordModification(latestSnapshotId);
176    removeAclFeature();
177    return this;
178  }
179
180  /**
181   * @param snapshotId
182   *          if it is not {@link Snapshot#CURRENT_STATE_ID}, get the result
183   *          from the given snapshot; otherwise, get the result from the
184   *          current inode.
185   * @return XAttrFeature
186   */  
187  abstract XAttrFeature getXAttrFeature(int snapshotId);
188  
189  @Override
190  public final XAttrFeature getXAttrFeature() {
191    return getXAttrFeature(Snapshot.CURRENT_STATE_ID);
192  }
193  
194  /**
195   * Set <code>XAttrFeature</code> 
196   */
197  abstract void addXAttrFeature(XAttrFeature xAttrFeature);
198  
199  final INode addXAttrFeature(XAttrFeature xAttrFeature, int latestSnapshotId) {
200    recordModification(latestSnapshotId);
201    addXAttrFeature(xAttrFeature);
202    return this;
203  }
204  
205  /**
206   * Remove <code>XAttrFeature</code> 
207   */
208  abstract void removeXAttrFeature();
209  
210  final INode removeXAttrFeature(int lastestSnapshotId) {
211    recordModification(lastestSnapshotId);
212    removeXAttrFeature();
213    return this;
214  }
215  
216  /**
217   * @return if the given snapshot id is {@link Snapshot#CURRENT_STATE_ID},
218   *         return this; otherwise return the corresponding snapshot inode.
219   */
220  public INodeAttributes getSnapshotINode(final int snapshotId) {
221    return this;
222  }
223
224  /** Is this inode in the latest snapshot? */
225  public final boolean isInLatestSnapshot(final int latestSnapshotId) {
226    if (latestSnapshotId == Snapshot.CURRENT_STATE_ID || latestSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
227      return false;
228    }
229    // if parent is a reference node, parent must be a renamed node. We can 
230    // stop the check at the reference node.
231    if (parent != null && parent.isReference()) {
232      return true;
233    }
234    final INodeDirectory parentDir = getParent();
235    if (parentDir == null) { // root
236      return true;
237    }
238    if (!parentDir.isInLatestSnapshot(latestSnapshotId)) {
239      return false;
240    }
241    final INode child = parentDir.getChild(getLocalNameBytes(), latestSnapshotId);
242    if (this == child) {
243      return true;
244    }
245    return child != null && child.isReference() &&
246        this == child.asReference().getReferredINode();
247  }
248  
249  /** @return true if the given inode is an ancestor directory of this inode. */
250  public final boolean isAncestorDirectory(final INodeDirectory dir) {
251    for(INodeDirectory p = getParent(); p != null; p = p.getParent()) {
252      if (p == dir) {
253        return true;
254      }
255    }
256    return false;
257  }
258
259  /**
260   * When {@link #recordModification} is called on a referred node,
261   * this method tells which snapshot the modification should be
262   * associated with: the snapshot that belongs to the SRC tree of the rename
263   * operation, or the snapshot belonging to the DST tree.
264   * 
265   * @param latestInDst
266   *          id of the latest snapshot in the DST tree above the reference node
267   * @return True: the modification should be recorded in the snapshot that
268   *         belongs to the SRC tree. False: the modification should be
269   *         recorded in the snapshot that belongs to the DST tree.
270   */
271  public final boolean shouldRecordInSrcSnapshot(final int latestInDst) {
272    Preconditions.checkState(!isReference());
273
274    if (latestInDst == Snapshot.CURRENT_STATE_ID) {
275      return true;
276    }
277    INodeReference withCount = getParentReference();
278    if (withCount != null) {
279      int dstSnapshotId = withCount.getParentReference().getDstSnapshotId();
280      if (dstSnapshotId != Snapshot.CURRENT_STATE_ID
281          && dstSnapshotId >= latestInDst) {
282        return true;
283      }
284    }
285    return false;
286  }
287
288  /**
289   * This inode is being modified.  The previous version of the inode needs to
290   * be recorded in the latest snapshot.
291   *
292   * @param latestSnapshotId The id of the latest snapshot that has been taken.
293   *                         Note that it is {@link Snapshot#CURRENT_STATE_ID} 
294   *                         if no snapshots have been taken.
295   */
296  abstract void recordModification(final int latestSnapshotId);
297
298  /** Check whether it's a reference. */
299  public boolean isReference() {
300    return false;
301  }
302
303  /** Cast this inode to an {@link INodeReference}.  */
304  public INodeReference asReference() {
305    throw new IllegalStateException("Current inode is not a reference: "
306        + this.toDetailString());
307  }
308
309  /**
310   * Check whether it's a file.
311   */
312  public boolean isFile() {
313    return false;
314  }
315
316  /** Cast this inode to an {@link INodeFile}.  */
317  public INodeFile asFile() {
318    throw new IllegalStateException("Current inode is not a file: "
319        + this.toDetailString());
320  }
321
322  /**
323   * Check whether it's a directory
324   */
325  public boolean isDirectory() {
326    return false;
327  }
328
329  /** Cast this inode to an {@link INodeDirectory}.  */
330  public INodeDirectory asDirectory() {
331    throw new IllegalStateException("Current inode is not a directory: "
332        + this.toDetailString());
333  }
334
335  /**
336   * Check whether it's a symlink
337   */
338  public boolean isSymlink() {
339    return false;
340  }
341
342  /** Cast this inode to an {@link INodeSymlink}.  */
343  public INodeSymlink asSymlink() {
344    throw new IllegalStateException("Current inode is not a symlink: "
345        + this.toDetailString());
346  }
347
348  /**
349   * Clean the subtree under this inode and collect the blocks from the descents
350   * for further block deletion/update. The current inode can either resides in
351   * the current tree or be stored as a snapshot copy.
352   * 
353   * <pre>
354   * In general, we have the following rules. 
355   * 1. When deleting a file/directory in the current tree, we have different 
356   * actions according to the type of the node to delete. 
357   * 
358   * 1.1 The current inode (this) is an {@link INodeFile}. 
359   * 1.1.1 If {@code prior} is null, there is no snapshot taken on ancestors 
360   * before. Thus we simply destroy (i.e., to delete completely, no need to save 
361   * snapshot copy) the current INode and collect its blocks for further 
362   * cleansing.
363   * 1.1.2 Else do nothing since the current INode will be stored as a snapshot
364   * copy.
365   * 
366   * 1.2 The current inode is an {@link INodeDirectory}.
367   * 1.2.1 If {@code prior} is null, there is no snapshot taken on ancestors 
368   * before. Similarly, we destroy the whole subtree and collect blocks.
369   * 1.2.2 Else do nothing with the current INode. Recursively clean its 
370   * children.
371   * 
372   * 1.3 The current inode is a file with snapshot.
373   * Call recordModification(..) to capture the current states.
374   * Mark the INode as deleted.
375   * 
376   * 1.4 The current inode is an {@link INodeDirectory} with snapshot feature.
377   * Call recordModification(..) to capture the current states. 
378   * Destroy files/directories created after the latest snapshot 
379   * (i.e., the inodes stored in the created list of the latest snapshot).
380   * Recursively clean remaining children. 
381   *
382   * 2. When deleting a snapshot.
383   * 2.1 To clean {@link INodeFile}: do nothing.
384   * 2.2 To clean {@link INodeDirectory}: recursively clean its children.
385   * 2.3 To clean INodeFile with snapshot: delete the corresponding snapshot in
386   * its diff list.
387   * 2.4 To clean {@link INodeDirectory} with snapshot: delete the corresponding 
388   * snapshot in its diff list. Recursively clean its children.
389   * </pre>
390   *
391   * @param bsps
392   *          block storage policy suite to calculate intended storage type usage
393   * @param snapshotId
394   *          The id of the snapshot to delete. 
395   *          {@link Snapshot#CURRENT_STATE_ID} means to delete the current
396   *          file/directory.
397   * @param priorSnapshotId
398   *          The id of the latest snapshot before the to-be-deleted snapshot.
399   *          When deleting a current inode, this parameter captures the latest
400   *          snapshot.
401   * @param collectedBlocks
402   *          blocks collected from the descents for further block
403   *          deletion/update will be added to the given map.
404   * @param removedINodes
405   *          INodes collected from the descents for further cleaning up of 
406   *          inodeMap
407   * @return quota usage delta when deleting a snapshot
408   */
409  public abstract QuotaCounts cleanSubtree(final BlockStoragePolicySuite bsps,
410      final int snapshotId,
411      int priorSnapshotId, BlocksMapUpdateInfo collectedBlocks,
412      List<INode> removedINodes);
413  
414  /**
415   * Destroy self and clear everything! If the INode is a file, this method
416   * collects its blocks for further block deletion. If the INode is a
417   * directory, the method goes down the subtree and collects blocks from the
418   * descents, and clears its parent/children references as well. The method
419   * also clears the diff list if the INode contains snapshot diff list.
420   *
421   * @param bsps
422   *          block storage policy suite to calculate intended storage type usage
423   *          This is needed because INodeReference#destroyAndCollectBlocks() needs
424   *          to call INode#cleanSubtree(), which calls INode#computeQuotaUsage().
425   * @param collectedBlocks
426   *          blocks collected from the descents for further block
427   *          deletion/update will be added to this map.
428   * @param removedINodes
429   *          INodes collected from the descents for further cleaning up of
430   *          inodeMap
431   */
432  public abstract void destroyAndCollectBlocks(
433      BlockStoragePolicySuite bsps,
434      BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes);
435
436  /** Compute {@link ContentSummary}. Blocking call */
437  public final ContentSummary computeContentSummary(BlockStoragePolicySuite bsps) {
438    return computeAndConvertContentSummary(
439        new ContentSummaryComputationContext(bsps));
440  }
441
442  /**
443   * Compute {@link ContentSummary}. 
444   */
445  public final ContentSummary computeAndConvertContentSummary(
446      ContentSummaryComputationContext summary) {
447    ContentCounts counts = computeContentSummary(summary).getCounts();
448    final QuotaCounts q = getQuotaCounts();
449    return new ContentSummary.Builder().
450        length(counts.getLength()).
451        fileCount(counts.getFileCount() + counts.getSymlinkCount()).
452        directoryCount(counts.getDirectoryCount()).
453        quota(q.getNameSpace()).
454        spaceConsumed(counts.getStoragespace()).
455        spaceQuota(q.getStorageSpace()).
456        typeConsumed(counts.getTypeSpaces()).
457        typeQuota(q.getTypeSpaces().asArray()).
458        build();
459  }
460
461  /**
462   * Count subtree content summary with a {@link ContentCounts}.
463   *
464   * @param summary the context object holding counts for the subtree.
465   * @return The same objects as summary.
466   */
467  public abstract ContentSummaryComputationContext computeContentSummary(
468      ContentSummaryComputationContext summary);
469
470
471  /**
472   * Check and add namespace/storagespace/storagetype consumed to itself and the ancestors.
473   * @throws QuotaExceededException if quote is violated.
474   */
475  public void addSpaceConsumed(QuotaCounts counts, boolean verify)
476    throws QuotaExceededException {
477    addSpaceConsumed2Parent(counts, verify);
478  }
479
480  /**
481   * Check and add namespace/storagespace/storagetype consumed to itself and the ancestors.
482   * @throws QuotaExceededException if quote is violated.
483   */
484  void addSpaceConsumed2Parent(QuotaCounts counts, boolean verify)
485    throws QuotaExceededException {
486    if (parent != null) {
487      parent.addSpaceConsumed(counts, verify);
488    }
489  }
490
491  /**
492   * Get the quota set for this inode
493   * @return the quota counts.  The count is -1 if it is not set.
494   */
495  public QuotaCounts getQuotaCounts() {
496    return new QuotaCounts.Builder().
497        nameSpace(HdfsConstants.QUOTA_RESET).
498        storageSpace(HdfsConstants.QUOTA_RESET).
499        typeSpaces(HdfsConstants.QUOTA_RESET).
500        build();
501  }
502
503  public final boolean isQuotaSet() {
504    final QuotaCounts qc = getQuotaCounts();
505    return qc.anyNsSsCountGreaterOrEqual(0) || qc.anyTypeSpaceCountGreaterOrEqual(0);
506  }
507
508  /**
509   * Count subtree {@link Quota#NAMESPACE} and {@link Quota#STORAGESPACE} usages.
510   * Entry point for FSDirectory where blockStoragePolicyId is given its initial
511   * value.
512   */
513  public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) {
514    final byte storagePolicyId = isSymlink() ?
515        BlockStoragePolicySuite.ID_UNSPECIFIED : getStoragePolicyID();
516    return computeQuotaUsage(bsps, storagePolicyId,
517        new QuotaCounts.Builder().build(), true, Snapshot.CURRENT_STATE_ID);
518  }
519
520  /**
521   * Count subtree {@link Quota#NAMESPACE} and {@link Quota#STORAGESPACE} usages.
522   * 
523   * With the existence of {@link INodeReference}, the same inode and its
524   * subtree may be referred by multiple {@link WithName} nodes and a
525   * {@link DstReference} node. To avoid circles while quota usage computation,
526   * we have the following rules:
527   * 
528   * <pre>
529   * 1. For a {@link DstReference} node, since the node must be in the current
530   * tree (or has been deleted as the end point of a series of rename 
531   * operations), we compute the quota usage of the referred node (and its 
532   * subtree) in the regular manner, i.e., including every inode in the current
533   * tree and in snapshot copies, as well as the size of diff list.
534   * 
535   * 2. For a {@link WithName} node, since the node must be in a snapshot, we 
536   * only count the quota usage for those nodes that still existed at the 
537   * creation time of the snapshot associated with the {@link WithName} node.
538   * We do not count in the size of the diff list.  
539   * <pre>
540   *
541   * @param bsps Block storage policy suite to calculate intended storage type usage
542   * @param blockStoragePolicyId block storage policy id of the current INode
543   * @param counts The subtree counts for returning.
544   * @param useCache Whether to use cached quota usage. Note that 
545   *                 {@link WithName} node never uses cache for its subtree.
546   * @param lastSnapshotId {@link Snapshot#CURRENT_STATE_ID} indicates the 
547   *                       computation is in the current tree. Otherwise the id
548   *                       indicates the computation range for a 
549   *                       {@link WithName} node.
550   * @return The same objects as the counts parameter.
551   */
552  public abstract QuotaCounts computeQuotaUsage(
553    BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
554    QuotaCounts counts, boolean useCache, int lastSnapshotId);
555
556  public final QuotaCounts computeQuotaUsage(
557    BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache) {
558    final byte storagePolicyId = isSymlink() ?
559        BlockStoragePolicySuite.ID_UNSPECIFIED : getStoragePolicyID();
560    return computeQuotaUsage(bsps, storagePolicyId, counts,
561        useCache, Snapshot.CURRENT_STATE_ID);
562  }
563
564  /**
565   * @return null if the local name is null; otherwise, return the local name.
566   */
567  public final String getLocalName() {
568    final byte[] name = getLocalNameBytes();
569    return name == null? null: DFSUtil.bytes2String(name);
570  }
571
572  @Override
573  public final byte[] getKey() {
574    return getLocalNameBytes();
575  }
576
577  /**
578   * Set local file name
579   */
580  public abstract void setLocalName(byte[] name);
581
582  public String getFullPathName() {
583    // Get the full path name of this inode.
584    if (isRoot()) {
585      return Path.SEPARATOR;
586    }
587    // compute size of needed bytes for the path
588    int idx = 0;
589    for (INode inode = this; inode != null; inode = inode.getParent()) {
590      // add component + delimiter (if not tail component)
591      idx += inode.getLocalNameBytes().length + (inode != this ? 1 : 0);
592    }
593    byte[] path = new byte[idx];
594    for (INode inode = this; inode != null; inode = inode.getParent()) {
595      if (inode != this) {
596        path[--idx] = Path.SEPARATOR_CHAR;
597      }
598      byte[] name = inode.getLocalNameBytes();
599      idx -= name.length;
600      System.arraycopy(name, 0, path, idx, name.length);
601    }
602    return DFSUtil.bytes2String(path);
603  }
604
605  public byte[][] getPathComponents() {
606    int n = 0;
607    for (INode inode = this; inode != null; inode = inode.getParent()) {
608      n++;
609    }
610    byte[][] components = new byte[n][];
611    for (INode inode = this; inode != null; inode = inode.getParent()) {
612      components[--n] = inode.getLocalNameBytes();
613    }
614    return components;
615  }
616
617  @Override
618  public String toString() {
619    return getLocalName();
620  }
621
622  @VisibleForTesting
623  public final String getObjectString() {
624    return getClass().getSimpleName() + "@"
625        + Integer.toHexString(super.hashCode());
626  }
627
628  /** @return a string description of the parent. */
629  @VisibleForTesting
630  public final String getParentString() {
631    final INodeReference parentRef = getParentReference();
632    if (parentRef != null) {
633      return "parentRef=" + parentRef.getLocalName() + "->";
634    } else {
635      final INodeDirectory parentDir = getParent();
636      if (parentDir != null) {
637        return "parentDir=" + parentDir.getLocalName() + "/";
638      } else {
639        return "parent=null";
640      }
641    }
642  }
643
644  @VisibleForTesting
645  public String toDetailString() {
646    return toString() + "(" + getObjectString() + "), " + getParentString();
647  }
648
649  /** @return the parent directory */
650  public final INodeDirectory getParent() {
651    return parent == null? null
652        : parent.isReference()? getParentReference().getParent(): parent.asDirectory();
653  }
654
655  /**
656   * @return the parent as a reference if this is a referred inode;
657   *         otherwise, return null.
658   */
659  public INodeReference getParentReference() {
660    return parent == null || !parent.isReference()? null: (INodeReference)parent;
661  }
662
663  /** Set parent directory */
664  public final void setParent(INodeDirectory parent) {
665    this.parent = parent;
666  }
667
668  /** Set container. */
669  public final void setParentReference(INodeReference parent) {
670    this.parent = parent;
671  }
672
673  /** Clear references to other objects. */
674  public void clear() {
675    setParent(null);
676  }
677
678  /**
679   * @param snapshotId
680   *          if it is not {@link Snapshot#CURRENT_STATE_ID}, get the result
681   *          from the given snapshot; otherwise, get the result from the
682   *          current inode.
683   * @return modification time.
684   */
685  abstract long getModificationTime(int snapshotId);
686
687  /** The same as getModificationTime(Snapshot.CURRENT_STATE_ID). */
688  @Override
689  public final long getModificationTime() {
690    return getModificationTime(Snapshot.CURRENT_STATE_ID);
691  }
692
693  /** Update modification time if it is larger than the current value. */
694  public abstract INode updateModificationTime(long mtime, int latestSnapshotId);
695
696  /** Set the last modification time of inode. */
697  public abstract void setModificationTime(long modificationTime);
698
699  /** Set the last modification time of inode. */
700  public final INode setModificationTime(long modificationTime,
701      int latestSnapshotId) {
702    recordModification(latestSnapshotId);
703    setModificationTime(modificationTime);
704    return this;
705  }
706
707  /**
708   * @param snapshotId
709   *          if it is not {@link Snapshot#CURRENT_STATE_ID}, get the result
710   *          from the given snapshot; otherwise, get the result from the
711   *          current inode.
712   * @return access time
713   */
714  abstract long getAccessTime(int snapshotId);
715
716  /** The same as getAccessTime(Snapshot.CURRENT_STATE_ID). */
717  @Override
718  public final long getAccessTime() {
719    return getAccessTime(Snapshot.CURRENT_STATE_ID);
720  }
721
722  /**
723   * Set last access time of inode.
724   */
725  public abstract void setAccessTime(long accessTime);
726
727  /**
728   * Set last access time of inode.
729   */
730  public final INode setAccessTime(long accessTime, int latestSnapshotId) {
731    recordModification(latestSnapshotId);
732    setAccessTime(accessTime);
733    return this;
734  }
735
736  /**
737   * @return the latest block storage policy id of the INode. Specifically,
738   * if a storage policy is directly specified on the INode then return the ID
739   * of that policy. Otherwise follow the latest parental path and return the
740   * ID of the first specified storage policy.
741   */
742  public abstract byte getStoragePolicyID();
743
744  /**
745   * @return the storage policy directly specified on the INode. Return
746   * {@link BlockStoragePolicySuite#ID_UNSPECIFIED} if no policy has
747   * been specified.
748   */
749  public abstract byte getLocalStoragePolicyID();
750
751  /**
752   * Get the storage policy ID while computing quota usage
753   * @param parentStoragePolicyId the storage policy ID of the parent directory
754   * @return the storage policy ID of this INode. Note that for an
755   * {@link INodeSymlink} we return {@link BlockStoragePolicySuite#ID_UNSPECIFIED}
756   * instead of throwing Exception
757   */
758  public byte getStoragePolicyIDForQuota(byte parentStoragePolicyId) {
759    byte localId = isSymlink() ?
760        BlockStoragePolicySuite.ID_UNSPECIFIED : getLocalStoragePolicyID();
761    return localId != BlockStoragePolicySuite.ID_UNSPECIFIED ?
762        localId : parentStoragePolicyId;
763  }
764
765  /**
766   * Breaks {@code path} into components.
767   * @return array of byte arrays each of which represents
768   * a single path component.
769   */
770  @VisibleForTesting
771  public static byte[][] getPathComponents(String path) {
772    checkAbsolutePath(path);
773    return DFSUtil.getPathComponents(path);
774  }
775
776  /**
777   * Splits an absolute {@code path} into an array of path components.
778   * @throws AssertionError if the given path is invalid.
779   * @return array of path components.
780   */
781  public static String[] getPathNames(String path) {
782    checkAbsolutePath(path);
783    return StringUtils.split(path, Path.SEPARATOR_CHAR);
784  }
785
786  private static void checkAbsolutePath(final String path) {
787    if (path == null || !path.startsWith(Path.SEPARATOR)) {
788      throw new AssertionError("Absolute path required");
789    }
790  }
791
792  @Override
793  public final int compareTo(byte[] bytes) {
794    return DFSUtil.compareBytes(getLocalNameBytes(), bytes);
795  }
796
797  @Override
798  public final boolean equals(Object that) {
799    if (this == that) {
800      return true;
801    }
802    if (that == null || !(that instanceof INode)) {
803      return false;
804    }
805    return getId() == ((INode) that).getId();
806  }
807
808  @Override
809  public final int hashCode() {
810    long id = getId();
811    return (int)(id^(id>>>32));  
812  }
813  
814  /**
815   * Dump the subtree starting from this inode.
816   * @return a text representation of the tree.
817   */
818  @VisibleForTesting
819  public final StringBuffer dumpTreeRecursively() {
820    final StringWriter out = new StringWriter(); 
821    dumpTreeRecursively(new PrintWriter(out, true), new StringBuilder(),
822        Snapshot.CURRENT_STATE_ID);
823    return out.getBuffer();
824  }
825
826  @VisibleForTesting
827  public final void dumpTreeRecursively(PrintStream out) {
828    out.println(dumpTreeRecursively().toString());
829  }
830
831  /**
832   * Dump tree recursively.
833   * @param prefix The prefix string that each line should print.
834   */
835  @VisibleForTesting
836  public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
837      int snapshotId) {
838    out.print(prefix);
839    out.print(" ");
840    final String name = getLocalName();
841    out.print(name.isEmpty()? "/": name);
842    out.print("   (");
843    out.print(getObjectString());
844    out.print("), ");
845    out.print(getParentString());
846    out.print(", " + getPermissionStatus(snapshotId));
847  }
848  
849  /**
850   * Information used for updating the blocksMap when deleting files.
851   */
852  public static class BlocksMapUpdateInfo {
853    /**
854     * The list of blocks that need to be removed from blocksMap
855     */
856    private final List<Block> toDeleteList;
857    
858    public BlocksMapUpdateInfo() {
859      toDeleteList = new ChunkedArrayList<Block>();
860    }
861    
862    /**
863     * @return The list of blocks that need to be removed from blocksMap
864     */
865    public List<Block> getToDeleteList() {
866      return toDeleteList;
867    }
868    
869    /**
870     * Add a to-be-deleted block into the
871     * {@link BlocksMapUpdateInfo#toDeleteList}
872     * @param toDelete the to-be-deleted block
873     */
874    public void addDeleteBlock(BlockInfoContiguous toDelete) {
875      assert toDelete != null : "toDelete is null";
876      toDeleteList.add(toDelete);
877      // If the file is being truncated
878      // the copy-on-truncate block should also be collected for deletion
879      if(!(toDelete instanceof BlockInfoContiguousUnderConstruction)) {
880        return;
881      }
882      BlockInfoContiguous truncateBlock =
883          ((BlockInfoContiguousUnderConstruction)toDelete).getTruncateBlock();
884      if(truncateBlock == null || truncateBlock.equals(toDelete)) {
885        return;
886      }
887      addDeleteBlock(truncateBlock);
888    }
889
890    public void removeDeleteBlock(Block block) {
891      assert block != null : "block is null";
892      toDeleteList.remove(block);
893    }
894
895    /**
896     * Clear {@link BlocksMapUpdateInfo#toDeleteList}
897     */
898    public void clear() {
899      toDeleteList.clear();
900    }
901  }
902
903  /** 
904   * INode feature such as {@link FileUnderConstructionFeature}
905   * and {@link DirectoryWithQuotaFeature}.
906   */
907  public interface Feature {
908  }
909}