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.server.blockmanagement.BlockStoragePolicySuite;
033import org.apache.hadoop.hdfs.DFSUtil;
034import org.apache.hadoop.hdfs.protocol.Block;
035import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
036import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
037import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
038import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
039import org.apache.hadoop.hdfs.util.ChunkedArrayList;
040import org.apache.hadoop.hdfs.util.Diff;
041import org.apache.hadoop.util.StringUtils;
042
043import com.google.common.annotations.VisibleForTesting;
044import 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
052public 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}