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