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) Support 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      public final short getFsPermissionShort() {
234        return referred.getFsPermissionShort();
235      }
236      
237      @Override
238      void setPermission(FsPermission permission) {
239        referred.setPermission(permission);
240      }
241    
242      @Override
243      public long getPermissionLong() {
244        return referred.getPermissionLong();
245      }
246    
247      @Override
248      public final long getModificationTime(int snapshotId) {
249        return referred.getModificationTime(snapshotId);
250      }
251      
252      @Override
253      public final INode updateModificationTime(long mtime, int latestSnapshotId) 
254          throws QuotaExceededException {
255        return referred.updateModificationTime(mtime, latestSnapshotId);
256      }
257      
258      @Override
259      public final void setModificationTime(long modificationTime) {
260        referred.setModificationTime(modificationTime);
261      }
262      
263      @Override
264      public final long getAccessTime(int snapshotId) {
265        return referred.getAccessTime(snapshotId);
266      }
267      
268      @Override
269      public final void setAccessTime(long accessTime) {
270        referred.setAccessTime(accessTime);
271      }
272    
273      @Override
274      final INode recordModification(int latestSnapshotId)
275          throws QuotaExceededException {
276        referred.recordModification(latestSnapshotId);
277        // reference is never replaced 
278        return this;
279      }
280    
281      @Override // used by WithCount
282      public Quota.Counts cleanSubtree(int snapshot, int prior,
283          BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes,
284          final boolean countDiffChange) throws QuotaExceededException {
285        return referred.cleanSubtree(snapshot, prior, collectedBlocks,
286            removedINodes, countDiffChange);
287      }
288    
289      @Override // used by WithCount
290      public void destroyAndCollectBlocks(
291          BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
292        if (removeReference(this) <= 0) {
293          referred.destroyAndCollectBlocks(collectedBlocks, removedINodes);
294        }
295      }
296    
297      @Override
298      public ContentSummaryComputationContext computeContentSummary(
299          ContentSummaryComputationContext summary) {
300        return referred.computeContentSummary(summary);
301      }
302    
303      @Override
304      public Quota.Counts computeQuotaUsage(Quota.Counts counts, boolean useCache,
305          int lastSnapshotId) {
306        return referred.computeQuotaUsage(counts, useCache, lastSnapshotId);
307      }
308      
309      @Override
310      public final INodeAttributes getSnapshotINode(int snapshotId) {
311        return referred.getSnapshotINode(snapshotId);
312      }
313    
314      @Override
315      public Quota.Counts getQuotaCounts() {
316        return referred.getQuotaCounts();
317      }
318    
319      @Override
320      public final void clear() {
321        super.clear();
322        referred = null;
323      }
324    
325      @Override
326      public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
327          final int snapshot) {
328        super.dumpTreeRecursively(out, prefix, snapshot);
329        if (this instanceof DstReference) {
330          out.print(", dstSnapshotId=" + ((DstReference) this).dstSnapshotId);
331        }
332        if (this instanceof WithCount) {
333          out.print(", count=" + ((WithCount)this).getReferenceCount());
334        }
335        out.println();
336        
337        final StringBuilder b = new StringBuilder();
338        for(int i = 0; i < prefix.length(); i++) {
339          b.append(' ');
340        }
341        b.append("->");
342        getReferredINode().dumpTreeRecursively(out, b, snapshot);
343      }
344      
345      public int getDstSnapshotId() {
346        return Snapshot.CURRENT_STATE_ID;
347      }
348      
349      /** An anonymous reference with reference count. */
350      public static class WithCount extends INodeReference {
351        
352        private final List<WithName> withNameList = new ArrayList<WithName>();
353        
354        /**
355         * Compare snapshot with IDs, where null indicates the current status thus
356         * is greater than any non-null snapshot.
357         */
358        public static final Comparator<WithName> WITHNAME_COMPARATOR
359            = new Comparator<WithName>() {
360          @Override
361          public int compare(WithName left, WithName right) {
362            return left.lastSnapshotId - right.lastSnapshotId;
363          }
364        };
365        
366        public WithCount(INodeReference parent, INode referred) {
367          super(parent, referred);
368          Preconditions.checkArgument(!referred.isReference());
369          referred.setParentReference(this);
370        }
371        
372        public int getReferenceCount() {
373          int count = withNameList.size();
374          if (getParentReference() != null) {
375            count++;
376          }
377          return count;
378        }
379    
380        /** Increment and then return the reference count. */
381        public void addReference(INodeReference ref) {
382          if (ref instanceof WithName) {
383            WithName refWithName = (WithName) ref;
384            int i = Collections.binarySearch(withNameList, refWithName,
385                WITHNAME_COMPARATOR);
386            Preconditions.checkState(i < 0);
387            withNameList.add(-i - 1, refWithName);
388          } else if (ref instanceof DstReference) {
389            setParentReference(ref);
390          }
391        }
392    
393        /** Decrement and then return the reference count. */
394        public void removeReference(INodeReference ref) {
395          if (ref instanceof WithName) {
396            int i = Collections.binarySearch(withNameList, (WithName) ref,
397                WITHNAME_COMPARATOR);
398            if (i >= 0) {
399              withNameList.remove(i);
400            }
401          } else if (ref == getParentReference()) {
402            setParent(null);
403          }
404        }
405        
406        WithName getLastWithName() {
407          return withNameList.size() > 0 ? 
408              withNameList.get(withNameList.size() - 1) : null;
409        }
410        
411        WithName getPriorWithName(WithName post) {
412          int i = Collections.binarySearch(withNameList, post, WITHNAME_COMPARATOR);
413          if (i > 0) {
414            return withNameList.get(i - 1);
415          } else if (i == 0 || i == -1) {
416            return null;
417          } else {
418            return withNameList.get(-i - 2);
419          }
420        }
421      }
422      
423      /** A reference with a fixed name. */
424      public static class WithName extends INodeReference {
425    
426        private final byte[] name;
427    
428        /**
429         * The id of the last snapshot in the src tree when this WithName node was 
430         * generated. When calculating the quota usage of the referred node, only 
431         * the files/dirs existing when this snapshot was taken will be counted for 
432         * this WithName node and propagated along its ancestor path.
433         */
434        private final int lastSnapshotId;
435        
436        public WithName(INodeDirectory parent, WithCount referred, byte[] name,
437            int lastSnapshotId) {
438          super(parent, referred);
439          this.name = name;
440          this.lastSnapshotId = lastSnapshotId;
441          referred.addReference(this);
442        }
443    
444        @Override
445        public final byte[] getLocalNameBytes() {
446          return name;
447        }
448    
449        @Override
450        public final void setLocalName(byte[] name) {
451          throw new UnsupportedOperationException("Cannot set name: " + getClass()
452              + " is immutable.");
453        }
454        
455        public int getLastSnapshotId() {
456          return lastSnapshotId;
457        }
458        
459        @Override
460        public final ContentSummaryComputationContext computeContentSummary(
461            ContentSummaryComputationContext summary) {
462          //only count diskspace for WithName
463          final Quota.Counts q = Quota.Counts.newInstance();
464          computeQuotaUsage(q, false, lastSnapshotId);
465          summary.getCounts().add(Content.DISKSPACE, q.get(Quota.DISKSPACE));
466          return summary;
467        }
468    
469        @Override
470        public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
471            boolean useCache, int lastSnapshotId) {
472          // if this.lastSnapshotId < lastSnapshotId, the rename of the referred 
473          // node happened before the rename of its ancestor. This should be 
474          // impossible since for WithName node we only count its children at the 
475          // time of the rename. 
476          Preconditions.checkState(lastSnapshotId == Snapshot.CURRENT_STATE_ID
477              || this.lastSnapshotId >= lastSnapshotId);
478          final INode referred = this.getReferredINode().asReference()
479              .getReferredINode();
480          // We will continue the quota usage computation using the same snapshot id
481          // as time line (if the given snapshot id is valid). Also, we cannot use 
482          // cache for the referred node since its cached quota may have already 
483          // been updated by changes in the current tree.
484          int id = lastSnapshotId != Snapshot.CURRENT_STATE_ID ? 
485              lastSnapshotId : this.lastSnapshotId;
486          return referred.computeQuotaUsage(counts, false, id);
487        }
488        
489        @Override
490        public Quota.Counts cleanSubtree(final int snapshot, int prior,
491            final BlocksMapUpdateInfo collectedBlocks,
492            final List<INode> removedINodes, final boolean countDiffChange)
493            throws QuotaExceededException {
494          // since WithName node resides in deleted list acting as a snapshot copy,
495          // the parameter snapshot must be non-null
496          Preconditions.checkArgument(snapshot != Snapshot.CURRENT_STATE_ID);
497          // if prior is NO_SNAPSHOT_ID, we need to check snapshot belonging to the
498          // previous WithName instance
499          if (prior == Snapshot.NO_SNAPSHOT_ID) {
500            prior = getPriorSnapshot(this);
501          }
502          
503          if (prior != Snapshot.NO_SNAPSHOT_ID
504              && Snapshot.ID_INTEGER_COMPARATOR.compare(snapshot, prior) <= 0) {
505            return Quota.Counts.newInstance();
506          }
507    
508          Quota.Counts counts = getReferredINode().cleanSubtree(snapshot, prior,
509              collectedBlocks, removedINodes, false);
510          INodeReference ref = getReferredINode().getParentReference();
511          if (ref != null) {
512            ref.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
513                -counts.get(Quota.DISKSPACE), true);
514          }
515          
516          if (snapshot < lastSnapshotId) {
517            // for a WithName node, when we compute its quota usage, we only count
518            // in all the nodes existing at the time of the corresponding rename op.
519            // Thus if we are deleting a snapshot before/at the snapshot associated 
520            // with lastSnapshotId, we do not need to update the quota upwards.
521            counts = Quota.Counts.newInstance();
522          }
523          return counts;
524        }
525        
526        @Override
527        public void destroyAndCollectBlocks(BlocksMapUpdateInfo collectedBlocks,
528            final List<INode> removedINodes) {
529          int snapshot = getSelfSnapshot();
530          if (removeReference(this) <= 0) {
531            getReferredINode().destroyAndCollectBlocks(collectedBlocks,
532                removedINodes);
533          } else {
534            int prior = getPriorSnapshot(this);
535            INode referred = getReferredINode().asReference().getReferredINode();
536            
537            if (snapshot != Snapshot.NO_SNAPSHOT_ID) {
538              if (prior != Snapshot.NO_SNAPSHOT_ID && snapshot <= prior) {
539                // the snapshot to be deleted has been deleted while traversing 
540                // the src tree of the previous rename operation. This usually 
541                // happens when rename's src and dst are under the same 
542                // snapshottable directory. E.g., the following operation sequence:
543                // 1. create snapshot s1 on /test
544                // 2. rename /test/foo/bar to /test/foo2/bar
545                // 3. create snapshot s2 on /test
546                // 4. rename foo2 again
547                // 5. delete snapshot s2
548                return;
549              }
550              try {
551                Quota.Counts counts = referred.cleanSubtree(snapshot, prior,
552                    collectedBlocks, removedINodes, false);
553                INodeReference ref = getReferredINode().getParentReference();
554                if (ref != null) {
555                  ref.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
556                      -counts.get(Quota.DISKSPACE), true);
557                }
558              } catch (QuotaExceededException e) {
559                LOG.error("should not exceed quota while snapshot deletion", e);
560              }
561            }
562          }
563        }
564        
565        private int getSelfSnapshot() {
566          INode referred = getReferredINode().asReference().getReferredINode();
567          int snapshot = Snapshot.NO_SNAPSHOT_ID;
568          if (referred.isFile() && referred.asFile().isWithSnapshot()) {
569            snapshot = referred.asFile().getDiffs().getPrior(lastSnapshotId);
570          } else if (referred.isDirectory()) {
571            DirectoryWithSnapshotFeature sf = referred.asDirectory()
572                .getDirectoryWithSnapshotFeature();
573            if (sf != null) {
574              snapshot = sf.getDiffs().getPrior(lastSnapshotId);
575            }
576          }
577          return snapshot;
578        }
579      }
580      
581      public static class DstReference extends INodeReference {
582        /**
583         * Record the latest snapshot of the dst subtree before the rename. For
584         * later operations on the moved/renamed files/directories, if the latest
585         * snapshot is after this dstSnapshot, changes will be recorded to the
586         * latest snapshot. Otherwise changes will be recorded to the snapshot
587         * belonging to the src of the rename.
588         * 
589         * {@link Snapshot#NO_SNAPSHOT_ID} means no dstSnapshot (e.g., src of the
590         * first-time rename).
591         */
592        private final int dstSnapshotId;
593        
594        @Override
595        public final int getDstSnapshotId() {
596          return dstSnapshotId;
597        }
598        
599        public DstReference(INodeDirectory parent, WithCount referred,
600            final int dstSnapshotId) {
601          super(parent, referred);
602          this.dstSnapshotId = dstSnapshotId;
603          referred.addReference(this);
604        }
605        
606        @Override
607        public Quota.Counts cleanSubtree(int snapshot, int prior,
608            BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes,
609            final boolean countDiffChange) throws QuotaExceededException {
610          if (snapshot == Snapshot.CURRENT_STATE_ID
611              && prior == Snapshot.NO_SNAPSHOT_ID) {
612            Quota.Counts counts = Quota.Counts.newInstance();
613            this.computeQuotaUsage(counts, true);
614            destroyAndCollectBlocks(collectedBlocks, removedINodes);
615            return counts;
616          } else {
617            // if prior is NO_SNAPSHOT_ID, we need to check snapshot belonging to 
618            // the previous WithName instance
619            if (prior == Snapshot.NO_SNAPSHOT_ID) {
620              prior = getPriorSnapshot(this);
621            }
622            // if prior is not NO_SNAPSHOT_ID, and prior is not before the
623            // to-be-deleted snapshot, we can quit here and leave the snapshot
624            // deletion work to the src tree of rename
625            if (snapshot != Snapshot.CURRENT_STATE_ID
626                && prior != Snapshot.NO_SNAPSHOT_ID
627                && Snapshot.ID_INTEGER_COMPARATOR.compare(snapshot, prior) <= 0) {
628              return Quota.Counts.newInstance();
629            }
630            return getReferredINode().cleanSubtree(snapshot, prior,
631                collectedBlocks, removedINodes, countDiffChange);
632          }
633        }
634        
635        /**
636         * {@inheritDoc}
637         * <br/>
638         * To destroy a DstReference node, we first remove its link with the 
639         * referred node. If the reference number of the referred node is <= 0, we 
640         * destroy the subtree of the referred node. Otherwise, we clean the 
641         * referred node's subtree and delete everything created after the last 
642         * rename operation, i.e., everything outside of the scope of the prior 
643         * WithName nodes.
644         */
645        @Override
646        public void destroyAndCollectBlocks(
647            BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
648          if (removeReference(this) <= 0) {
649            getReferredINode().destroyAndCollectBlocks(collectedBlocks,
650                removedINodes);
651          } else {
652            // we will clean everything, including files, directories, and 
653            // snapshots, that were created after this prior snapshot
654            int prior = getPriorSnapshot(this);
655            // prior must be non-null, otherwise we do not have any previous 
656            // WithName nodes, and the reference number will be 0.
657            Preconditions.checkState(prior != Snapshot.NO_SNAPSHOT_ID);
658            // identify the snapshot created after prior
659            int snapshot = getSelfSnapshot(prior);
660            
661            INode referred = getReferredINode().asReference().getReferredINode();
662            if (referred.isFile()) {
663              // if referred is a file, it must be a file with snapshot since we did
664              // recordModification before the rename
665              INodeFile file = referred.asFile();
666              Preconditions.checkState(file.isWithSnapshot());
667              // make sure we mark the file as deleted
668              file.getFileWithSnapshotFeature().deleteCurrentFile();
669              try {
670                // when calling cleanSubtree of the referred node, since we 
671                // compute quota usage updates before calling this destroy 
672                // function, we use true for countDiffChange
673                referred.cleanSubtree(snapshot, prior, collectedBlocks,
674                    removedINodes, true);
675              } catch (QuotaExceededException e) {
676                LOG.error("should not exceed quota while snapshot deletion", e);
677              }
678            } else if (referred.isDirectory()) {
679              // similarly, if referred is a directory, it must be an
680              // INodeDirectory with snapshot
681              INodeDirectory dir = referred.asDirectory();
682              Preconditions.checkState(dir.isWithSnapshot());
683              try {
684                DirectoryWithSnapshotFeature.destroyDstSubtree(dir, snapshot,
685                    prior, collectedBlocks, removedINodes);
686              } catch (QuotaExceededException e) {
687                LOG.error("should not exceed quota while snapshot deletion", e);
688              }
689            }
690          }
691        }
692        
693        private int getSelfSnapshot(final int prior) {
694          WithCount wc = (WithCount) getReferredINode().asReference();
695          INode referred = wc.getReferredINode();
696          int lastSnapshot = Snapshot.CURRENT_STATE_ID;
697          if (referred.isFile() && referred.asFile().isWithSnapshot()) {
698            lastSnapshot = referred.asFile().getDiffs().getLastSnapshotId();
699          } else if (referred.isDirectory()) {
700            DirectoryWithSnapshotFeature sf = referred.asDirectory()
701                .getDirectoryWithSnapshotFeature();
702            if (sf != null) {
703              lastSnapshot = sf.getLastSnapshotId();
704            }
705          }
706          if (lastSnapshot != Snapshot.CURRENT_STATE_ID && lastSnapshot != prior) {
707            return lastSnapshot;
708          } else {
709            return Snapshot.CURRENT_STATE_ID;
710          }
711        }
712      }
713    }