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