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