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     */
018    package org.apache.hadoop.hdfs.server.namenode;
019    
020    import java.io.PrintWriter;
021    import java.util.ArrayList;
022    import java.util.Collections;
023    import java.util.Comparator;
024    import java.util.List;
025    
026    import org.apache.hadoop.fs.permission.FsPermission;
027    import org.apache.hadoop.fs.permission.PermissionStatus;
028    import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
029    import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
030    import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
031    
032    import 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     */
062    public 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    }