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