1 /**
2  * Licensed to the Apache Software Foundation (ASF) under one
3  * or more contributor license agreements.  See the NOTICE file
4  * distributed with this work for additional information
5  * regarding copyright ownership.  The ASF licenses this file
6  * to you under the Apache License, Version 2.0 (the
7  * "License"); you may not use this file except in compliance
8  * with the License.  You may obtain a copy of the License at
9  *
10  *     http://www.apache.org/licenses/LICENSE-2.0
11  *
12  * Unless required by applicable law or agreed to in writing, software
13  * distributed under the License is distributed on an "AS IS" BASIS,
14  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15  * See the License for the specific language governing permissions and
16  * limitations under the License.
17  */
18 package org.apache.hadoop.hdfs.server.namenode;
19 
20 import java.io.PrintStream;
21 import java.io.PrintWriter;
22 import java.io.StringWriter;
23 import java.util.List;
24 
25 import org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27 import org.apache.hadoop.classification.InterfaceAudience;
28 import org.apache.hadoop.fs.ContentSummary;
29 import org.apache.hadoop.fs.Path;
30 import org.apache.hadoop.fs.permission.FsPermission;
31 import org.apache.hadoop.fs.permission.PermissionStatus;
32 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
33 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
34 import org.apache.hadoop.hdfs.DFSUtil;
35 import org.apache.hadoop.hdfs.protocol.Block;
36 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
37 import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
38 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
39 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
40 import org.apache.hadoop.hdfs.util.Diff;
41 import org.apache.hadoop.util.ChunkedArrayList;
42 import org.apache.hadoop.util.StringUtils;
43 
44 import com.google.common.annotations.VisibleForTesting;
45 import com.google.common.base.Preconditions;
46 
47 /**
48  * We keep an in-memory representation of the file/block hierarchy.
49  * This is a base INode class containing common fields for file and
50  * directory inodes.
51  */
52 @InterfaceAudience.Private
53 public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
54   public static final Log LOG = LogFactory.getLog(INode.class);
55 
56   /** parent is either an {@link INodeDirectory} or an {@link INodeReference}.*/
57   private INode parent = null;
58 
INode(INode parent)59   INode(INode parent) {
60     this.parent = parent;
61   }
62 
63   /** Get inode id */
getId()64   public abstract long getId();
65 
66   /**
67    * Check whether this is the root inode.
68    */
isRoot()69   final boolean isRoot() {
70     return getLocalNameBytes().length == 0;
71   }
72 
73   /** Get the {@link PermissionStatus} */
getPermissionStatus(int snapshotId)74   abstract PermissionStatus getPermissionStatus(int snapshotId);
75 
76   /** The same as getPermissionStatus(null). */
getPermissionStatus()77   final PermissionStatus getPermissionStatus() {
78     return getPermissionStatus(Snapshot.CURRENT_STATE_ID);
79   }
80 
81   /**
82    * @param snapshotId
83    *          if it is not {@link Snapshot#CURRENT_STATE_ID}, get the result
84    *          from the given snapshot; otherwise, get the result from the
85    *          current inode.
86    * @return user name
87    */
getUserName(int snapshotId)88   abstract String getUserName(int snapshotId);
89 
90   /** The same as getUserName(Snapshot.CURRENT_STATE_ID). */
91   @Override
getUserName()92   public final String getUserName() {
93     return getUserName(Snapshot.CURRENT_STATE_ID);
94   }
95 
96   /** Set user */
setUser(String user)97   abstract void setUser(String user);
98 
99   /** Set user */
setUser(String user, int latestSnapshotId)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    */
getGroupName(int snapshotId)112   abstract String getGroupName(int snapshotId);
113 
114   /** The same as getGroupName(Snapshot.CURRENT_STATE_ID). */
115   @Override
getGroupName()116   public final String getGroupName() {
117     return getGroupName(Snapshot.CURRENT_STATE_ID);
118   }
119 
120   /** Set group */
setGroup(String group)121   abstract void setGroup(String group);
122 
123   /** Set group */
setGroup(String group, int latestSnapshotId)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    */
getFsPermission(int snapshotId)137   abstract FsPermission getFsPermission(int snapshotId);
138 
139   /** The same as getFsPermission(Snapshot.CURRENT_STATE_ID). */
140   @Override
getFsPermission()141   public final FsPermission getFsPermission() {
142     return getFsPermission(Snapshot.CURRENT_STATE_ID);
143   }
144 
145   /** Set the {@link FsPermission} of this {@link INode} */
setPermission(FsPermission permission)146   abstract void setPermission(FsPermission permission);
147 
148   /** Set the {@link FsPermission} of this {@link INode} */
setPermission(FsPermission permission, int latestSnapshotId)149   INode setPermission(FsPermission permission, int latestSnapshotId) {
150     recordModification(latestSnapshotId);
151     setPermission(permission);
152     return this;
153   }
154 
getAclFeature(int snapshotId)155   abstract AclFeature getAclFeature(int snapshotId);
156 
157   @Override
getAclFeature()158   public final AclFeature getAclFeature() {
159     return getAclFeature(Snapshot.CURRENT_STATE_ID);
160   }
161 
addAclFeature(AclFeature aclFeature)162   abstract void addAclFeature(AclFeature aclFeature);
163 
addAclFeature(AclFeature aclFeature, int latestSnapshotId)164   final INode addAclFeature(AclFeature aclFeature, int latestSnapshotId) {
165     recordModification(latestSnapshotId);
166     addAclFeature(aclFeature);
167     return this;
168   }
169 
removeAclFeature()170   abstract void removeAclFeature();
171 
removeAclFeature(int latestSnapshotId)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    */
getXAttrFeature(int snapshotId)185   abstract XAttrFeature getXAttrFeature(int snapshotId);
186 
187   @Override
getXAttrFeature()188   public final XAttrFeature getXAttrFeature() {
189     return getXAttrFeature(Snapshot.CURRENT_STATE_ID);
190   }
191 
192   /**
193    * Set <code>XAttrFeature</code>
194    */
addXAttrFeature(XAttrFeature xAttrFeature)195   abstract void addXAttrFeature(XAttrFeature xAttrFeature);
196 
addXAttrFeature(XAttrFeature xAttrFeature, int latestSnapshotId)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    */
removeXAttrFeature()206   abstract void removeXAttrFeature();
207 
removeXAttrFeature(int lastestSnapshotId)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    */
getSnapshotINode(final int snapshotId)218   public INodeAttributes getSnapshotINode(final int snapshotId) {
219     return this;
220   }
221 
222   /** Is this inode in the latest snapshot? */
isInLatestSnapshot(final int latestSnapshotId)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. */
isAncestorDirectory(final INodeDirectory dir)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    */
shouldRecordInSrcSnapshot(final int latestInDst)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    */
recordModification(final int latestSnapshotId)294   abstract void recordModification(final int latestSnapshotId);
295 
296   /** Check whether it's a reference. */
isReference()297   public boolean isReference() {
298     return false;
299   }
300 
301   /** Cast this inode to an {@link INodeReference}.  */
asReference()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    */
isFile()310   public boolean isFile() {
311     return false;
312   }
313 
314   /** Cast this inode to an {@link INodeFile}.  */
asFile()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    */
isDirectory()323   public boolean isDirectory() {
324     return false;
325   }
326 
327   /** Cast this inode to an {@link INodeDirectory}.  */
asDirectory()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    */
isSymlink()336   public boolean isSymlink() {
337     return false;
338   }
339 
340   /** Cast this inode to an {@link INodeSymlink}.  */
asSymlink()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    */
cleanSubtree(final BlockStoragePolicySuite bsps, final int snapshotId, int priorSnapshotId, BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes)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    */
destroyAndCollectBlocks( BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes)430   public abstract void destroyAndCollectBlocks(
431       BlockStoragePolicySuite bsps,
432       BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes);
433 
434   /** Compute {@link ContentSummary}. Blocking call */
computeContentSummary(BlockStoragePolicySuite bsps)435   public final ContentSummary computeContentSummary(BlockStoragePolicySuite bsps) {
436     return computeAndConvertContentSummary(
437         new ContentSummaryComputationContext(bsps));
438   }
439 
440   /**
441    * Compute {@link ContentSummary}.
442    */
computeAndConvertContentSummary( ContentSummaryComputationContext summary)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    */
computeContentSummary( ContentSummaryComputationContext summary)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    */
addSpaceConsumed(QuotaCounts counts, boolean verify)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    */
addSpaceConsumed2Parent(QuotaCounts counts, boolean verify)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    */
getQuotaCounts()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 
isQuotaSet()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    */
computeQuotaUsage(BlockStoragePolicySuite bsps)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    */
computeQuotaUsage( BlockStoragePolicySuite bsps, byte blockStoragePolicyId, QuotaCounts counts, boolean useCache, int lastSnapshotId)550   public abstract QuotaCounts computeQuotaUsage(
551     BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
552     QuotaCounts counts, boolean useCache, int lastSnapshotId);
553 
computeQuotaUsage( BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache)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    */
getLocalName()565   public final String getLocalName() {
566     final byte[] name = getLocalNameBytes();
567     return name == null? null: DFSUtil.bytes2String(name);
568   }
569 
570   @Override
getKey()571   public final byte[] getKey() {
572     return getLocalNameBytes();
573   }
574 
575   /**
576    * Set local file name
577    */
setLocalName(byte[] name)578   public abstract void setLocalName(byte[] name);
579 
getFullPathName()580   public String getFullPathName() {
581     // Get the full path name of this inode.
582     return FSDirectory.getFullPathName(this);
583   }
584 
585   @Override
toString()586   public String toString() {
587     return getLocalName();
588   }
589 
590   @VisibleForTesting
getObjectString()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
getParentString()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
toDetailString()613   public String toDetailString() {
614     return toString() + "(" + getObjectString() + "), " + getParentString();
615   }
616 
617   /** @return the parent directory */
getParent()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    */
getParentReference()627   public INodeReference getParentReference() {
628     return parent == null || !parent.isReference()? null: (INodeReference)parent;
629   }
630 
631   /** Set parent directory */
setParent(INodeDirectory parent)632   public final void setParent(INodeDirectory parent) {
633     this.parent = parent;
634   }
635 
636   /** Set container. */
setParentReference(INodeReference parent)637   public final void setParentReference(INodeReference parent) {
638     this.parent = parent;
639   }
640 
641   /** Clear references to other objects. */
clear()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    */
getModificationTime(int snapshotId)653   abstract long getModificationTime(int snapshotId);
654 
655   /** The same as getModificationTime(Snapshot.CURRENT_STATE_ID). */
656   @Override
getModificationTime()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. */
updateModificationTime(long mtime, int latestSnapshotId)662   public abstract INode updateModificationTime(long mtime, int latestSnapshotId);
663 
664   /** Set the last modification time of inode. */
setModificationTime(long modificationTime)665   public abstract void setModificationTime(long modificationTime);
666 
667   /** Set the last modification time of inode. */
setModificationTime(long modificationTime, int latestSnapshotId)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    */
getAccessTime(int snapshotId)682   abstract long getAccessTime(int snapshotId);
683 
684   /** The same as getAccessTime(Snapshot.CURRENT_STATE_ID). */
685   @Override
getAccessTime()686   public final long getAccessTime() {
687     return getAccessTime(Snapshot.CURRENT_STATE_ID);
688   }
689 
690   /**
691    * Set last access time of inode.
692    */
setAccessTime(long accessTime)693   public abstract void setAccessTime(long accessTime);
694 
695   /**
696    * Set last access time of inode.
697    */
setAccessTime(long accessTime, int latestSnapshotId)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    */
getStoragePolicyID()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    */
getLocalStoragePolicyID()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    */
getStoragePolicyIDForQuota(byte parentStoragePolicyId)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
getPathComponents(String path)739   public static byte[][] getPathComponents(String path) {
740     return getPathComponents(getPathNames(path));
741   }
742 
743   /** Convert strings to byte arrays for path components. */
getPathComponents(String[] strings)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    */
getPathNames(String path)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
compareTo(byte[] bytes)767   public final int compareTo(byte[] bytes) {
768     return DFSUtil.compareBytes(getLocalNameBytes(), bytes);
769   }
770 
771   @Override
equals(Object that)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
hashCode()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
dumpTreeRecursively()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
dumpTreeRecursively(PrintStream out)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
dumpTreeRecursively(PrintWriter out, StringBuilder prefix, int snapshotId)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 
BlocksMapUpdateInfo()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      */
getToDeleteList()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      */
addDeleteBlock(Block toDelete)848     public void addDeleteBlock(Block toDelete) {
849       assert toDelete != null : "toDelete is null";
850       toDeleteList.add(toDelete);
851     }
852 
removeDeleteBlock(Block block)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      */
clear()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 }
873