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