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.PrintWriter;
021import java.util.ArrayList;
022import java.util.Collections;
023import java.util.Comparator;
024import java.util.List;
025
026import org.apache.hadoop.fs.permission.FsPermission;
027import org.apache.hadoop.fs.permission.PermissionStatus;
028import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
029import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
030import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
031import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
032
033import com.google.common.base.Preconditions;
034import org.mortbay.log.Log;
035
036/**
037 * An anonymous reference to an inode.
038 *
039 * This class and its subclasses are used to support multiple access paths.
040 * A file/directory may have multiple access paths when it is stored in some
041 * snapshots and it is renamed/moved to other locations.
042 * 
043 * For example,
044 * (1) Suppose we have /abc/foo, say the inode of foo is inode(id=1000,name=foo)
045 * (2) create snapshot s0 for /abc
046 * (3) mv /abc/foo /xyz/bar, i.e. inode(id=1000,name=...) is renamed from "foo"
047 *     to "bar" and its parent becomes /xyz.
048 * 
049 * Then, /xyz/bar and /abc/.snapshot/s0/foo are two different access paths to
050 * the same inode, inode(id=1000,name=bar).
051 *
052 * With references, we have the following
053 * - /abc has a child ref(id=1001,name=foo).
054 * - /xyz has a child ref(id=1002) 
055 * - Both ref(id=1001,name=foo) and ref(id=1002) point to another reference,
056 *   ref(id=1003,count=2).
057 * - Finally, ref(id=1003,count=2) points to inode(id=1000,name=bar).
058 * 
059 * Note 1: For a reference without name, e.g. ref(id=1002), it uses the name
060 *         of the referred inode.
061 * Note 2: getParent() always returns the parent in the current state, e.g.
062 *         inode(id=1000,name=bar).getParent() returns /xyz but not /abc.
063 */
064public abstract class INodeReference extends INode {
065  /**
066   * Try to remove the given reference and then return the reference count.
067   * If the given inode is not a reference, return -1;
068   */
069  public static int tryRemoveReference(INode inode) {
070    if (!inode.isReference()) {
071      return -1;
072    }
073    return removeReference(inode.asReference());
074  }
075
076  /**
077   * Remove the given reference and then return the reference count.
078   * If the referred inode is not a WithCount, return -1;
079   */
080  private static int removeReference(INodeReference ref) {
081    final INode referred = ref.getReferredINode();
082    if (!(referred instanceof WithCount)) {
083      return -1;
084    }
085    
086    WithCount wc = (WithCount) referred;
087    wc.removeReference(ref);
088    return wc.getReferenceCount();
089  }
090
091  /**
092   * When destroying a reference node (WithName or DstReference), we call this
093   * method to identify the snapshot which is the latest snapshot before the
094   * reference node's creation. 
095   */
096  static int getPriorSnapshot(INodeReference ref) {
097    WithCount wc = (WithCount) ref.getReferredINode();
098    WithName wn = null;
099    if (ref instanceof DstReference) {
100      wn = wc.getLastWithName();
101    } else if (ref instanceof WithName) {
102      wn = wc.getPriorWithName((WithName) ref);
103    }
104    if (wn != null) {
105      INode referred = wc.getReferredINode();
106      if (referred.isFile() && referred.asFile().isWithSnapshot()) {
107        return referred.asFile().getDiffs().getPrior(wn.lastSnapshotId);
108      } else if (referred.isDirectory()) {
109        DirectoryWithSnapshotFeature sf = referred.asDirectory()
110            .getDirectoryWithSnapshotFeature();
111        if (sf != null) {
112          return sf.getDiffs().getPrior(wn.lastSnapshotId);
113        }
114      }
115    }
116    return Snapshot.NO_SNAPSHOT_ID;
117  }
118  
119  private INode referred;
120  
121  public INodeReference(INode parent, INode referred) {
122    super(parent);
123    this.referred = referred;
124  }
125
126  public final INode getReferredINode() {
127    return referred;
128  }
129
130  public final void setReferredINode(INode referred) {
131    this.referred = referred;
132  }
133  
134  @Override
135  public final boolean isReference() {
136    return true;
137  }
138  
139  @Override
140  public final INodeReference asReference() {
141    return this;
142  }
143
144  @Override
145  public final boolean isFile() {
146    return referred.isFile();
147  }
148  
149  @Override
150  public final INodeFile asFile() {
151    return referred.asFile();
152  }
153  
154  @Override
155  public final boolean isDirectory() {
156    return referred.isDirectory();
157  }
158  
159  @Override
160  public final INodeDirectory asDirectory() {
161    return referred.asDirectory();
162  }
163  
164  @Override
165  public final boolean isSymlink() {
166    return referred.isSymlink();
167  }
168  
169  @Override
170  public final INodeSymlink asSymlink() {
171    return referred.asSymlink();
172  }
173
174  @Override
175  public byte[] getLocalNameBytes() {
176    return referred.getLocalNameBytes();
177  }
178
179  @Override
180  public void setLocalName(byte[] name) {
181    referred.setLocalName(name);
182  }
183
184  @Override
185  public final long getId() {
186    return referred.getId();
187  }
188  
189  @Override
190  public final PermissionStatus getPermissionStatus(int snapshotId) {
191    return referred.getPermissionStatus(snapshotId);
192  }
193  
194  @Override
195  public final String getUserName(int snapshotId) {
196    return referred.getUserName(snapshotId);
197  }
198  
199  @Override
200  final void setUser(String user) {
201    referred.setUser(user);
202  }
203  
204  @Override
205  public final String getGroupName(int snapshotId) {
206    return referred.getGroupName(snapshotId);
207  }
208  
209  @Override
210  final void setGroup(String group) {
211    referred.setGroup(group);
212  }
213  
214  @Override
215  public final FsPermission getFsPermission(int snapshotId) {
216    return referred.getFsPermission(snapshotId);
217  }
218
219  @Override
220  final AclFeature getAclFeature(int snapshotId) {
221    return referred.getAclFeature(snapshotId);
222  }
223
224  @Override
225  final void addAclFeature(AclFeature aclFeature) {
226    referred.addAclFeature(aclFeature);
227  }
228
229  @Override
230  final void removeAclFeature() {
231    referred.removeAclFeature();
232  }
233  
234  @Override
235  final XAttrFeature getXAttrFeature(int snapshotId) {
236    return referred.getXAttrFeature(snapshotId);
237  }
238  
239  @Override
240  final void addXAttrFeature(XAttrFeature xAttrFeature) {
241    referred.addXAttrFeature(xAttrFeature);
242  }
243  
244  @Override
245  final void removeXAttrFeature() {
246    referred.removeXAttrFeature();
247  }
248
249  @Override
250  public final short getFsPermissionShort() {
251    return referred.getFsPermissionShort();
252  }
253  
254  @Override
255  void setPermission(FsPermission permission) {
256    referred.setPermission(permission);
257  }
258
259  @Override
260  public long getPermissionLong() {
261    return referred.getPermissionLong();
262  }
263
264  @Override
265  public final long getModificationTime(int snapshotId) {
266    return referred.getModificationTime(snapshotId);
267  }
268  
269  @Override
270  public final INode updateModificationTime(long mtime, int latestSnapshotId) {
271    return referred.updateModificationTime(mtime, latestSnapshotId);
272  }
273  
274  @Override
275  public final void setModificationTime(long modificationTime) {
276    referred.setModificationTime(modificationTime);
277  }
278  
279  @Override
280  public final long getAccessTime(int snapshotId) {
281    return referred.getAccessTime(snapshotId);
282  }
283  
284  @Override
285  public final void setAccessTime(long accessTime) {
286    referred.setAccessTime(accessTime);
287  }
288
289  @Override
290  public final byte getStoragePolicyID() {
291    return referred.getStoragePolicyID();
292  }
293
294  @Override
295  public final byte getLocalStoragePolicyID() {
296    return referred.getLocalStoragePolicyID();
297  }
298
299  @Override
300  final void recordModification(int latestSnapshotId) {
301    referred.recordModification(latestSnapshotId);
302  }
303
304  @Override // used by WithCount
305  public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, int snapshot,
306      int prior, BlocksMapUpdateInfo collectedBlocks,
307      final List<INode> removedINodes) {
308    return referred.cleanSubtree(bsps, snapshot, prior, collectedBlocks,
309        removedINodes);
310  }
311
312  @Override // used by WithCount
313  public void destroyAndCollectBlocks(
314      BlockStoragePolicySuite bsps,
315      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
316    if (removeReference(this) <= 0) {
317      referred.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes);
318    }
319  }
320
321  @Override
322  public ContentSummaryComputationContext computeContentSummary(
323      ContentSummaryComputationContext summary) {
324    return referred.computeContentSummary(summary);
325  }
326
327  @Override
328  public QuotaCounts computeQuotaUsage(
329    BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
330    QuotaCounts counts, boolean useCache, int lastSnapshotId) {
331    return referred.computeQuotaUsage(bsps, blockStoragePolicyId, counts,
332        useCache, lastSnapshotId);
333  }
334
335  @Override
336  public final INodeAttributes getSnapshotINode(int snapshotId) {
337    return referred.getSnapshotINode(snapshotId);
338  }
339
340  @Override
341  public QuotaCounts getQuotaCounts() {
342    return referred.getQuotaCounts();
343  }
344
345  @Override
346  public final void clear() {
347    super.clear();
348    referred = null;
349  }
350
351  @Override
352  public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
353      final int snapshot) {
354    super.dumpTreeRecursively(out, prefix, snapshot);
355    if (this instanceof DstReference) {
356      out.print(", dstSnapshotId=" + ((DstReference) this).dstSnapshotId);
357    }
358    if (this instanceof WithCount) {
359      out.print(", count=" + ((WithCount)this).getReferenceCount());
360    }
361    out.println();
362    
363    final StringBuilder b = new StringBuilder();
364    for(int i = 0; i < prefix.length(); i++) {
365      b.append(' ');
366    }
367    b.append("->");
368    getReferredINode().dumpTreeRecursively(out, b, snapshot);
369  }
370  
371  public int getDstSnapshotId() {
372    return Snapshot.CURRENT_STATE_ID;
373  }
374  
375  /** An anonymous reference with reference count. */
376  public static class WithCount extends INodeReference {
377    
378    private final List<WithName> withNameList = new ArrayList<WithName>();
379    
380    /**
381     * Compare snapshot with IDs, where null indicates the current status thus
382     * is greater than any non-null snapshot.
383     */
384    public static final Comparator<WithName> WITHNAME_COMPARATOR
385        = new Comparator<WithName>() {
386      @Override
387      public int compare(WithName left, WithName right) {
388        return left.lastSnapshotId - right.lastSnapshotId;
389      }
390    };
391    
392    public WithCount(INodeReference parent, INode referred) {
393      super(parent, referred);
394      Preconditions.checkArgument(!referred.isReference());
395      referred.setParentReference(this);
396    }
397    
398    public int getReferenceCount() {
399      int count = withNameList.size();
400      if (getParentReference() != null) {
401        count++;
402      }
403      return count;
404    }
405
406    /** Increment and then return the reference count. */
407    public void addReference(INodeReference ref) {
408      if (ref instanceof WithName) {
409        WithName refWithName = (WithName) ref;
410        int i = Collections.binarySearch(withNameList, refWithName,
411            WITHNAME_COMPARATOR);
412        Preconditions.checkState(i < 0);
413        withNameList.add(-i - 1, refWithName);
414      } else if (ref instanceof DstReference) {
415        setParentReference(ref);
416      }
417    }
418
419    /** Decrement and then return the reference count. */
420    public void removeReference(INodeReference ref) {
421      if (ref instanceof WithName) {
422        int i = Collections.binarySearch(withNameList, (WithName) ref,
423            WITHNAME_COMPARATOR);
424        if (i >= 0) {
425          withNameList.remove(i);
426        }
427      } else if (ref == getParentReference()) {
428        setParent(null);
429      }
430    }
431    
432    WithName getLastWithName() {
433      return withNameList.size() > 0 ? 
434          withNameList.get(withNameList.size() - 1) : null;
435    }
436    
437    WithName getPriorWithName(WithName post) {
438      int i = Collections.binarySearch(withNameList, post, WITHNAME_COMPARATOR);
439      if (i > 0) {
440        return withNameList.get(i - 1);
441      } else if (i == 0 || i == -1) {
442        return null;
443      } else {
444        return withNameList.get(-i - 2);
445      }
446    }
447
448    /**
449     * @return the WithName/DstReference node contained in the given snapshot.
450     */
451    public INodeReference getParentRef(int snapshotId) {
452      int start = 0;
453      int end = withNameList.size() - 1;
454      while (start < end) {
455        int mid = start + (end - start) / 2;
456        int sid = withNameList.get(mid).lastSnapshotId; 
457        if (sid == snapshotId) {
458          return withNameList.get(mid);
459        } else if (sid < snapshotId) {
460          start = mid + 1;
461        } else {
462          end = mid;
463        }
464      }
465      if (start < withNameList.size() &&
466          withNameList.get(start).lastSnapshotId >= snapshotId) {
467        return withNameList.get(start);
468      } else {
469        return this.getParentReference();
470      }
471    }
472  }
473  
474  /** A reference with a fixed name. */
475  public static class WithName extends INodeReference {
476
477    private final byte[] name;
478
479    /**
480     * The id of the last snapshot in the src tree when this WithName node was 
481     * generated. When calculating the quota usage of the referred node, only 
482     * the files/dirs existing when this snapshot was taken will be counted for 
483     * this WithName node and propagated along its ancestor path.
484     */
485    private final int lastSnapshotId;
486    
487    public WithName(INodeDirectory parent, WithCount referred, byte[] name,
488        int lastSnapshotId) {
489      super(parent, referred);
490      this.name = name;
491      this.lastSnapshotId = lastSnapshotId;
492      referred.addReference(this);
493    }
494
495    @Override
496    public final byte[] getLocalNameBytes() {
497      return name;
498    }
499
500    @Override
501    public final void setLocalName(byte[] name) {
502      throw new UnsupportedOperationException("Cannot set name: " + getClass()
503          + " is immutable.");
504    }
505    
506    public int getLastSnapshotId() {
507      return lastSnapshotId;
508    }
509    
510    @Override
511    public final ContentSummaryComputationContext computeContentSummary(
512        ContentSummaryComputationContext summary) {
513      //only count storagespace for WithName
514      final QuotaCounts q = new QuotaCounts.Builder().build();
515      computeQuotaUsage(summary.getBlockStoragePolicySuite(),
516          getStoragePolicyID(), q, false, lastSnapshotId);
517      summary.getCounts().addContent(Content.DISKSPACE, q.getStorageSpace());
518      summary.getCounts().addTypeSpaces(q.getTypeSpaces());
519      return summary;
520    }
521
522    @Override
523    public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
524        byte blockStoragePolicyId, QuotaCounts counts, boolean useCache,
525        int lastSnapshotId) {
526      // if this.lastSnapshotId < lastSnapshotId, the rename of the referred 
527      // node happened before the rename of its ancestor. This should be 
528      // impossible since for WithName node we only count its children at the 
529      // time of the rename. 
530      Preconditions.checkState(lastSnapshotId == Snapshot.CURRENT_STATE_ID
531          || this.lastSnapshotId >= lastSnapshotId);
532      final INode referred = this.getReferredINode().asReference()
533          .getReferredINode();
534      // We will continue the quota usage computation using the same snapshot id
535      // as time line (if the given snapshot id is valid). Also, we cannot use 
536      // cache for the referred node since its cached quota may have already 
537      // been updated by changes in the current tree.
538      int id = lastSnapshotId != Snapshot.CURRENT_STATE_ID ? 
539          lastSnapshotId : this.lastSnapshotId;
540      return referred.computeQuotaUsage(bsps, blockStoragePolicyId, counts,
541          false, id);
542    }
543    
544    @Override
545    public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps,
546        final int snapshot, int prior, final BlocksMapUpdateInfo collectedBlocks,
547        final List<INode> removedINodes) {
548      // since WithName node resides in deleted list acting as a snapshot copy,
549      // the parameter snapshot must be non-null
550      Preconditions.checkArgument(snapshot != Snapshot.CURRENT_STATE_ID);
551      // if prior is NO_SNAPSHOT_ID, we need to check snapshot belonging to the
552      // previous WithName instance
553      if (prior == Snapshot.NO_SNAPSHOT_ID) {
554        prior = getPriorSnapshot(this);
555      }
556      
557      if (prior != Snapshot.NO_SNAPSHOT_ID
558          && Snapshot.ID_INTEGER_COMPARATOR.compare(snapshot, prior) <= 0) {
559        return new QuotaCounts.Builder().build();
560      }
561
562      QuotaCounts counts = getReferredINode().cleanSubtree(bsps, snapshot, prior,
563          collectedBlocks, removedINodes);
564      INodeReference ref = getReferredINode().getParentReference();
565      if (ref != null) {
566        try {
567          ref.addSpaceConsumed(counts.negation(), true);
568        } catch (QuotaExceededException e) {
569          Log.warn("Should not have QuotaExceededException");
570        }
571      }
572      
573      if (snapshot < lastSnapshotId) {
574        // for a WithName node, when we compute its quota usage, we only count
575        // in all the nodes existing at the time of the corresponding rename op.
576        // Thus if we are deleting a snapshot before/at the snapshot associated 
577        // with lastSnapshotId, we do not need to update the quota upwards.
578        counts = new QuotaCounts.Builder().build();
579      }
580      return counts;
581    }
582    
583    @Override
584    public void destroyAndCollectBlocks(BlockStoragePolicySuite bsps,
585        BlocksMapUpdateInfo collectedBlocks,
586        final List<INode> removedINodes) {
587      int snapshot = getSelfSnapshot();
588      if (removeReference(this) <= 0) {
589        getReferredINode().destroyAndCollectBlocks(bsps, collectedBlocks,
590            removedINodes);
591      } else {
592        int prior = getPriorSnapshot(this);
593        INode referred = getReferredINode().asReference().getReferredINode();
594        
595        if (snapshot != Snapshot.NO_SNAPSHOT_ID) {
596          if (prior != Snapshot.NO_SNAPSHOT_ID && snapshot <= prior) {
597            // the snapshot to be deleted has been deleted while traversing 
598            // the src tree of the previous rename operation. This usually 
599            // happens when rename's src and dst are under the same 
600            // snapshottable directory. E.g., the following operation sequence:
601            // 1. create snapshot s1 on /test
602            // 2. rename /test/foo/bar to /test/foo2/bar
603            // 3. create snapshot s2 on /test
604            // 4. rename foo2 again
605            // 5. delete snapshot s2
606            return;
607          }
608          try {
609            QuotaCounts counts = referred.cleanSubtree(bsps, snapshot, prior,
610                collectedBlocks, removedINodes);
611            INodeReference ref = getReferredINode().getParentReference();
612            if (ref != null) {
613              ref.addSpaceConsumed(counts.negation(), true);
614            }
615          } catch (QuotaExceededException e) {
616            LOG.error("should not exceed quota while snapshot deletion", e);
617          }
618        }
619      }
620    }
621    
622    private int getSelfSnapshot() {
623      INode referred = getReferredINode().asReference().getReferredINode();
624      int snapshot = Snapshot.NO_SNAPSHOT_ID;
625      if (referred.isFile() && referred.asFile().isWithSnapshot()) {
626        snapshot = referred.asFile().getDiffs().getPrior(lastSnapshotId);
627      } else if (referred.isDirectory()) {
628        DirectoryWithSnapshotFeature sf = referred.asDirectory()
629            .getDirectoryWithSnapshotFeature();
630        if (sf != null) {
631          snapshot = sf.getDiffs().getPrior(lastSnapshotId);
632        }
633      }
634      return snapshot;
635    }
636  }
637  
638  public static class DstReference extends INodeReference {
639    /**
640     * Record the latest snapshot of the dst subtree before the rename. For
641     * later operations on the moved/renamed files/directories, if the latest
642     * snapshot is after this dstSnapshot, changes will be recorded to the
643     * latest snapshot. Otherwise changes will be recorded to the snapshot
644     * belonging to the src of the rename.
645     * 
646     * {@link Snapshot#NO_SNAPSHOT_ID} means no dstSnapshot (e.g., src of the
647     * first-time rename).
648     */
649    private final int dstSnapshotId;
650    
651    @Override
652    public final int getDstSnapshotId() {
653      return dstSnapshotId;
654    }
655    
656    public DstReference(INodeDirectory parent, WithCount referred,
657        final int dstSnapshotId) {
658      super(parent, referred);
659      this.dstSnapshotId = dstSnapshotId;
660      referred.addReference(this);
661    }
662    
663    @Override
664    public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, int snapshot, int prior,
665        BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes) {
666      if (snapshot == Snapshot.CURRENT_STATE_ID
667          && prior == Snapshot.NO_SNAPSHOT_ID) {
668        QuotaCounts counts = new QuotaCounts.Builder().build();
669        this.computeQuotaUsage(bsps, counts, true);
670        destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes);
671        return counts;
672      } else {
673        // if prior is NO_SNAPSHOT_ID, we need to check snapshot belonging to 
674        // the previous WithName instance
675        if (prior == Snapshot.NO_SNAPSHOT_ID) {
676          prior = getPriorSnapshot(this);
677        }
678        // if prior is not NO_SNAPSHOT_ID, and prior is not before the
679        // to-be-deleted snapshot, we can quit here and leave the snapshot
680        // deletion work to the src tree of rename
681        if (snapshot != Snapshot.CURRENT_STATE_ID
682            && prior != Snapshot.NO_SNAPSHOT_ID
683            && Snapshot.ID_INTEGER_COMPARATOR.compare(snapshot, prior) <= 0) {
684          return new QuotaCounts.Builder().build();
685        }
686        return getReferredINode().cleanSubtree(bsps, snapshot, prior,
687            collectedBlocks, removedINodes);
688      }
689    }
690    
691    /**
692     * {@inheritDoc}
693     * <br/>
694     * To destroy a DstReference node, we first remove its link with the 
695     * referred node. If the reference number of the referred node is <= 0, we 
696     * destroy the subtree of the referred node. Otherwise, we clean the 
697     * referred node's subtree and delete everything created after the last 
698     * rename operation, i.e., everything outside of the scope of the prior 
699     * WithName nodes.
700     */
701    @Override
702    public void destroyAndCollectBlocks(BlockStoragePolicySuite bsps,
703        BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
704      if (removeReference(this) <= 0) {
705        getReferredINode().destroyAndCollectBlocks(bsps, collectedBlocks,
706            removedINodes);
707      } else {
708        // we will clean everything, including files, directories, and 
709        // snapshots, that were created after this prior snapshot
710        int prior = getPriorSnapshot(this);
711        // prior must be non-null, otherwise we do not have any previous 
712        // WithName nodes, and the reference number will be 0.
713        Preconditions.checkState(prior != Snapshot.NO_SNAPSHOT_ID);
714        // identify the snapshot created after prior
715        int snapshot = getSelfSnapshot(prior);
716        
717        INode referred = getReferredINode().asReference().getReferredINode();
718        if (referred.isFile()) {
719          // if referred is a file, it must be a file with snapshot since we did
720          // recordModification before the rename
721          INodeFile file = referred.asFile();
722          Preconditions.checkState(file.isWithSnapshot());
723          // make sure we mark the file as deleted
724          file.getFileWithSnapshotFeature().deleteCurrentFile();
725          // when calling cleanSubtree of the referred node, since we
726          // compute quota usage updates before calling this destroy
727          // function, we use true for countDiffChange
728          referred.cleanSubtree(bsps, snapshot, prior, collectedBlocks,
729              removedINodes);
730        } else if (referred.isDirectory()) {
731          // similarly, if referred is a directory, it must be an
732          // INodeDirectory with snapshot
733          INodeDirectory dir = referred.asDirectory();
734          Preconditions.checkState(dir.isWithSnapshot());
735          try {
736            DirectoryWithSnapshotFeature.destroyDstSubtree(bsps, dir, snapshot,
737                prior, collectedBlocks, removedINodes);
738          } catch (QuotaExceededException e) {
739            LOG.error("should not exceed quota while snapshot deletion", e);
740          }
741        }
742      }
743    }
744
745    private int getSelfSnapshot(final int prior) {
746      WithCount wc = (WithCount) getReferredINode().asReference();
747      INode referred = wc.getReferredINode();
748      int lastSnapshot = Snapshot.CURRENT_STATE_ID;
749      if (referred.isFile() && referred.asFile().isWithSnapshot()) {
750        lastSnapshot = referred.asFile().getDiffs().getLastSnapshotId();
751      } else if (referred.isDirectory()) {
752        DirectoryWithSnapshotFeature sf = referred.asDirectory()
753            .getDirectoryWithSnapshotFeature();
754        if (sf != null) {
755          lastSnapshot = sf.getLastSnapshotId();
756        }
757      }
758      if (lastSnapshot != Snapshot.CURRENT_STATE_ID && lastSnapshot != prior) {
759        return lastSnapshot;
760      } else {
761        return Snapshot.CURRENT_STATE_ID;
762      }
763    }
764  }
765}