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