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.snapshot;
019
020import java.io.DataOutput;
021import java.io.IOException;
022import java.util.ArrayDeque;
023import java.util.ArrayList;
024import java.util.Collections;
025import java.util.Deque;
026import java.util.HashMap;
027import java.util.Iterator;
028import java.util.List;
029import java.util.Map;
030
031import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
032import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
033import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
034import org.apache.hadoop.hdfs.server.namenode.Content;
035import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
036import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
037import org.apache.hadoop.hdfs.server.namenode.INode;
038import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
039import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
040import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
041import org.apache.hadoop.hdfs.server.namenode.INodeFile;
042import org.apache.hadoop.hdfs.server.namenode.INodeMap;
043import org.apache.hadoop.hdfs.server.namenode.INodeReference;
044import org.apache.hadoop.hdfs.server.namenode.Quota;
045import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
046import org.apache.hadoop.hdfs.util.Diff;
047import org.apache.hadoop.hdfs.util.Diff.Container;
048import org.apache.hadoop.hdfs.util.Diff.ListType;
049import org.apache.hadoop.hdfs.util.Diff.UndoInfo;
050import org.apache.hadoop.hdfs.util.ReadOnlyList;
051
052import com.google.common.base.Preconditions;
053
054/**
055 * The directory with snapshots. It maintains a list of snapshot diffs for
056 * storing snapshot data. When there are modifications to the directory, the old
057 * data is stored in the latest snapshot, if there is any.
058 */
059public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
060  /**
061   * The difference between the current state and a previous snapshot
062   * of the children list of an INodeDirectory.
063   */
064  static class ChildrenDiff extends Diff<byte[], INode> {
065    ChildrenDiff() {}
066    
067    private ChildrenDiff(final List<INode> created, final List<INode> deleted) {
068      super(created, deleted);
069    }
070
071    /**
072     * Replace the given child from the created/deleted list.
073     * @return true if the child is replaced; false if the child is not found.
074     */
075    private final boolean replace(final ListType type,
076        final INode oldChild, final INode newChild) {
077      final List<INode> list = getList(type); 
078      final int i = search(list, oldChild.getLocalNameBytes());
079      if (i < 0 || list.get(i).getId() != oldChild.getId()) {
080        return false;
081      }
082
083      final INode removed = list.set(i, newChild);
084      Preconditions.checkState(removed == oldChild);
085      return true;
086    }
087
088    private final boolean removeChild(ListType type, final INode child) {
089      final List<INode> list = getList(type);
090      final int i = searchIndex(type, child.getLocalNameBytes());
091      if (i >= 0 && list.get(i) == child) {
092        list.remove(i);
093        return true;
094      }
095      return false;
096    }
097    
098    /** clear the created list */
099    private Quota.Counts destroyCreatedList(
100        final INodeDirectoryWithSnapshot currentINode,
101        final BlocksMapUpdateInfo collectedBlocks,
102        final List<INode> removedINodes) {
103      Quota.Counts counts = Quota.Counts.newInstance();
104      final List<INode> createdList = getList(ListType.CREATED);
105      for (INode c : createdList) {
106        c.computeQuotaUsage(counts, true);
107        c.destroyAndCollectBlocks(collectedBlocks, removedINodes);
108        // c should be contained in the children list, remove it
109        currentINode.removeChild(c);
110      }
111      createdList.clear();
112      return counts;
113    }
114    
115    /** clear the deleted list */
116    private Quota.Counts destroyDeletedList(
117        final BlocksMapUpdateInfo collectedBlocks,
118        final List<INode> removedINodes) {
119      Quota.Counts counts = Quota.Counts.newInstance();
120      final List<INode> deletedList = getList(ListType.DELETED);
121      for (INode d : deletedList) {
122        d.computeQuotaUsage(counts, false);
123        d.destroyAndCollectBlocks(collectedBlocks, removedINodes);
124      }
125      deletedList.clear();
126      return counts;
127    }
128    
129    /** Serialize {@link #created} */
130    private void writeCreated(DataOutput out) throws IOException {
131      final List<INode> created = getList(ListType.CREATED);
132      out.writeInt(created.size());
133      for (INode node : created) {
134        // For INode in created list, we only need to record its local name 
135        byte[] name = node.getLocalNameBytes();
136        out.writeShort(name.length);
137        out.write(name);
138      }
139    }
140    
141    /** Serialize {@link #deleted} */
142    private void writeDeleted(DataOutput out,
143        ReferenceMap referenceMap) throws IOException {
144      final List<INode> deleted = getList(ListType.DELETED);
145      out.writeInt(deleted.size());
146      for (INode node : deleted) {
147        FSImageSerialization.saveINode2Image(node, out, true, referenceMap);
148      }
149    }
150    
151    /** Serialize to out */
152    private void write(DataOutput out, ReferenceMap referenceMap
153        ) throws IOException {
154      writeCreated(out);
155      writeDeleted(out, referenceMap);    
156    }
157
158    /** Get the list of INodeDirectory contained in the deleted list */
159    private void getDirsInDeleted(List<INodeDirectory> dirList) {
160      for (INode node : getList(ListType.DELETED)) {
161        if (node.isDirectory()) {
162          dirList.add(node.asDirectory());
163        }
164      }
165    }
166    
167    /**
168     * Interpret the diff and generate a list of {@link DiffReportEntry}.
169     * @param parentPath The relative path of the parent.
170     * @param parent The directory that the diff belongs to.
171     * @param fromEarlier True indicates {@code diff=later-earlier}, 
172     *                    False indicates {@code diff=earlier-later}
173     * @return A list of {@link DiffReportEntry} as the diff report.
174     */
175    public List<DiffReportEntry> generateReport(byte[][] parentPath,
176        INodeDirectoryWithSnapshot parent, boolean fromEarlier) {
177      List<DiffReportEntry> cList = new ArrayList<DiffReportEntry>();
178      List<DiffReportEntry> dList = new ArrayList<DiffReportEntry>();
179      int c = 0, d = 0;
180      List<INode> created = getList(ListType.CREATED);
181      List<INode> deleted = getList(ListType.DELETED);
182      byte[][] fullPath = new byte[parentPath.length + 1][];
183      System.arraycopy(parentPath, 0, fullPath, 0, parentPath.length);
184      for (; c < created.size() && d < deleted.size(); ) {
185        INode cnode = created.get(c);
186        INode dnode = deleted.get(d);
187        if (cnode.compareTo(dnode.getLocalNameBytes()) == 0) {
188          fullPath[fullPath.length - 1] = cnode.getLocalNameBytes();
189          // must be the case: delete first and then create an inode with the
190          // same name
191          cList.add(new DiffReportEntry(DiffType.CREATE, fullPath));
192          dList.add(new DiffReportEntry(DiffType.DELETE, fullPath));
193          c++;
194          d++;
195        } else if (cnode.compareTo(dnode.getLocalNameBytes()) < 0) {
196          fullPath[fullPath.length - 1] = cnode.getLocalNameBytes();
197          cList.add(new DiffReportEntry(fromEarlier ? DiffType.CREATE
198              : DiffType.DELETE, fullPath));
199          c++;
200        } else {
201          fullPath[fullPath.length - 1] = dnode.getLocalNameBytes();
202          dList.add(new DiffReportEntry(fromEarlier ? DiffType.DELETE
203              : DiffType.CREATE, fullPath));
204          d++;
205        }
206      }
207      for (; d < deleted.size(); d++) {
208        fullPath[fullPath.length - 1] = deleted.get(d).getLocalNameBytes();
209        dList.add(new DiffReportEntry(fromEarlier ? DiffType.DELETE
210            : DiffType.CREATE, fullPath));
211      }
212      for (; c < created.size(); c++) {
213        fullPath[fullPath.length - 1] = created.get(c).getLocalNameBytes();
214        cList.add(new DiffReportEntry(fromEarlier ? DiffType.CREATE
215            : DiffType.DELETE, fullPath));
216      }
217      dList.addAll(cList);
218      return dList;
219    }
220  }
221  
222  /**
223   * The difference of an {@link INodeDirectory} between two snapshots.
224   */
225  public static class DirectoryDiff extends
226      AbstractINodeDiff<INodeDirectory, INodeDirectoryAttributes, DirectoryDiff> {
227    /** The size of the children list at snapshot creation time. */
228    private final int childrenSize;
229    /** The children list diff. */
230    private final ChildrenDiff diff;
231
232    private DirectoryDiff(Snapshot snapshot, INodeDirectory dir) {
233      super(snapshot, null, null);
234
235      this.childrenSize = dir.getChildrenList(null).size();
236      this.diff = new ChildrenDiff();
237    }
238
239    /** Constructor used by FSImage loading */
240    DirectoryDiff(Snapshot snapshot, INodeDirectoryAttributes snapshotINode,
241        DirectoryDiff posteriorDiff, int childrenSize,
242        List<INode> createdList, List<INode> deletedList) {
243      super(snapshot, snapshotINode, posteriorDiff);
244      this.childrenSize = childrenSize;
245      this.diff = new ChildrenDiff(createdList, deletedList);
246    }
247    
248    ChildrenDiff getChildrenDiff() {
249      return diff;
250    }
251    
252    /** Is the inode the root of the snapshot? */
253    boolean isSnapshotRoot() {
254      return snapshotINode == snapshot.getRoot();
255    }
256    
257    @Override
258    Quota.Counts combinePosteriorAndCollectBlocks(
259        final INodeDirectory currentDir, final DirectoryDiff posterior,
260        final BlocksMapUpdateInfo collectedBlocks,
261        final List<INode> removedINodes) {
262      final Quota.Counts counts = Quota.Counts.newInstance();
263      diff.combinePosterior(posterior.diff, new Diff.Processor<INode>() {
264        /** Collect blocks for deleted files. */
265        @Override
266        public void process(INode inode) {
267          if (inode != null) {
268            inode.computeQuotaUsage(counts, false);
269            inode.destroyAndCollectBlocks(collectedBlocks, removedINodes);
270          }
271        }
272      });
273      return counts;
274    }
275
276    /**
277     * @return The children list of a directory in a snapshot.
278     *         Since the snapshot is read-only, the logical view of the list is
279     *         never changed although the internal data structure may mutate.
280     */
281    ReadOnlyList<INode> getChildrenList(final INodeDirectory currentDir) {
282      return new ReadOnlyList<INode>() {
283        private List<INode> children = null;
284
285        private List<INode> initChildren() {
286          if (children == null) {
287            final ChildrenDiff combined = new ChildrenDiff();
288            for(DirectoryDiff d = DirectoryDiff.this; d != null; d = d.getPosterior()) {
289              combined.combinePosterior(d.diff, null);
290            }
291            children = combined.apply2Current(ReadOnlyList.Util.asList(
292                currentDir.getChildrenList(null)));
293          }
294          return children;
295        }
296
297        @Override
298        public Iterator<INode> iterator() {
299          return initChildren().iterator();
300        }
301    
302        @Override
303        public boolean isEmpty() {
304          return childrenSize == 0;
305        }
306    
307        @Override
308        public int size() {
309          return childrenSize;
310        }
311    
312        @Override
313        public INode get(int i) {
314          return initChildren().get(i);
315        }
316      };
317    }
318
319    /** @return the child with the given name. */
320    INode getChild(byte[] name, boolean checkPosterior,
321        INodeDirectory currentDir) {
322      for(DirectoryDiff d = this; ; d = d.getPosterior()) {
323        final Container<INode> returned = d.diff.accessPrevious(name);
324        if (returned != null) {
325          // the diff is able to determine the inode
326          return returned.getElement(); 
327        } else if (!checkPosterior) {
328          // Since checkPosterior is false, return null, i.e. not found.   
329          return null;
330        } else if (d.getPosterior() == null) {
331          // no more posterior diff, get from current inode.
332          return currentDir.getChild(name, null);
333        }
334      }
335    }
336    
337    @Override
338    public String toString() {
339      return super.toString() + " childrenSize=" + childrenSize + ", " + diff;
340    }
341    
342    @Override
343    void write(DataOutput out, ReferenceMap referenceMap) throws IOException {
344      writeSnapshot(out);
345      out.writeInt(childrenSize);
346
347      // write snapshotINode
348      if (isSnapshotRoot()) {
349        out.writeBoolean(true);
350      } else {
351        out.writeBoolean(false);
352        if (snapshotINode != null) {
353          out.writeBoolean(true);
354          FSImageSerialization.writeINodeDirectoryAttributes(snapshotINode, out);
355        } else {
356          out.writeBoolean(false);
357        }
358      }
359      // Write diff. Node need to write poseriorDiff, since diffs is a list.
360      diff.write(out, referenceMap);
361    }
362
363    @Override
364    Quota.Counts destroyDiffAndCollectBlocks(INodeDirectory currentINode,
365        BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
366      // this diff has been deleted
367      Quota.Counts counts = Quota.Counts.newInstance();
368      counts.add(diff.destroyDeletedList(collectedBlocks, removedINodes));
369      return counts;
370    }
371  }
372
373  /** A list of directory diffs. */
374  public static class DirectoryDiffList
375      extends AbstractINodeDiffList<INodeDirectory, INodeDirectoryAttributes, DirectoryDiff> {
376
377    @Override
378    DirectoryDiff createDiff(Snapshot snapshot, INodeDirectory currentDir) {
379      return new DirectoryDiff(snapshot, currentDir);
380    }
381
382    @Override
383    INodeDirectoryAttributes createSnapshotCopy(INodeDirectory currentDir) {
384      return currentDir.isQuotaSet()?
385          new INodeDirectoryAttributes.CopyWithQuota(currentDir)
386        : new INodeDirectoryAttributes.SnapshotCopy(currentDir);
387    }
388
389    /** Replace the given child in the created/deleted list, if there is any. */
390    private boolean replaceChild(final ListType type, final INode oldChild,
391        final INode newChild) {
392      final List<DirectoryDiff> diffList = asList();
393      for(int i = diffList.size() - 1; i >= 0; i--) {
394        final ChildrenDiff diff = diffList.get(i).diff;
395        if (diff.replace(type, oldChild, newChild)) {
396          return true;
397        }
398      }
399      return false;
400    }
401    
402    /** Remove the given child in the created/deleted list, if there is any. */
403    private boolean removeChild(final ListType type, final INode child) {
404      final List<DirectoryDiff> diffList = asList();
405      for(int i = diffList.size() - 1; i >= 0; i--) {
406        final ChildrenDiff diff = diffList.get(i).diff;
407        if (diff.removeChild(type, child)) {
408          return true;
409        }
410      }
411      return false;
412    }
413  }
414
415  /**
416   * Compute the difference between Snapshots.
417   * 
418   * @param fromSnapshot Start point of the diff computation. Null indicates
419   *          current tree.
420   * @param toSnapshot End point of the diff computation. Null indicates current
421   *          tree.
422   * @param diff Used to capture the changes happening to the children. Note
423   *          that the diff still represents (later_snapshot - earlier_snapshot)
424   *          although toSnapshot can be before fromSnapshot.
425   * @return Whether changes happened between the startSnapshot and endSnaphsot.
426   */
427  boolean computeDiffBetweenSnapshots(Snapshot fromSnapshot,
428      Snapshot toSnapshot, ChildrenDiff diff) {
429    Snapshot earlier = fromSnapshot;
430    Snapshot later = toSnapshot;
431    if (Snapshot.ID_COMPARATOR.compare(fromSnapshot, toSnapshot) > 0) {
432      earlier = toSnapshot;
433      later = fromSnapshot;
434    }
435    
436    boolean modified = diffs.changedBetweenSnapshots(earlier,
437        later);
438    if (!modified) {
439      return false;
440    }
441    
442    final List<DirectoryDiff> difflist = diffs.asList();
443    final int size = difflist.size();
444    int earlierDiffIndex = Collections.binarySearch(difflist, earlier.getId());
445    int laterDiffIndex = later == null ? size : Collections
446        .binarySearch(difflist, later.getId());
447    earlierDiffIndex = earlierDiffIndex < 0 ? (-earlierDiffIndex - 1)
448        : earlierDiffIndex;
449    laterDiffIndex = laterDiffIndex < 0 ? (-laterDiffIndex - 1)
450        : laterDiffIndex;
451    
452    boolean dirMetadataChanged = false;
453    INodeDirectoryAttributes dirCopy = null;
454    for (int i = earlierDiffIndex; i < laterDiffIndex; i++) {
455      DirectoryDiff sdiff = difflist.get(i);
456      diff.combinePosterior(sdiff.diff, null);
457      if (dirMetadataChanged == false && sdiff.snapshotINode != null) {
458        if (dirCopy == null) {
459          dirCopy = sdiff.snapshotINode;
460        } else if (!dirCopy.metadataEquals(sdiff.snapshotINode)) {
461          dirMetadataChanged = true;
462        }
463      }
464    }
465
466    if (!diff.isEmpty() || dirMetadataChanged) {
467      return true;
468    } else if (dirCopy != null) {
469      for (int i = laterDiffIndex; i < size; i++) {
470        if (!dirCopy.metadataEquals(difflist.get(i).snapshotINode)) {
471          return true;
472        }
473      }
474      return !dirCopy.metadataEquals(this);
475    } else {
476      return false;
477    }
478  }
479
480  /** Diff list sorted by snapshot IDs, i.e. in chronological order. */
481  private final DirectoryDiffList diffs;
482
483  public INodeDirectoryWithSnapshot(INodeDirectory that) {
484    this(that, true, that instanceof INodeDirectoryWithSnapshot?
485        ((INodeDirectoryWithSnapshot)that).getDiffs(): null);
486  }
487
488  INodeDirectoryWithSnapshot(INodeDirectory that, boolean adopt,
489      DirectoryDiffList diffs) {
490    super(that, adopt, that.getNsQuota(), that.getDsQuota());
491    this.diffs = diffs != null? diffs: new DirectoryDiffList();
492  }
493
494  /** @return the last snapshot. */
495  public Snapshot getLastSnapshot() {
496    return diffs.getLastSnapshot();
497  }
498
499  /** @return the snapshot diff list. */
500  public DirectoryDiffList getDiffs() {
501    return diffs;
502  }
503
504  @Override
505  public INodeDirectoryAttributes getSnapshotINode(Snapshot snapshot) {
506    return diffs.getSnapshotINode(snapshot, this);
507  }
508
509  @Override
510  public INodeDirectoryWithSnapshot recordModification(final Snapshot latest,
511      final INodeMap inodeMap) throws QuotaExceededException {
512    if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
513      return saveSelf2Snapshot(latest, null);
514    }
515    return this;
516  }
517
518  /** Save the snapshot copy to the latest snapshot. */
519  public INodeDirectoryWithSnapshot saveSelf2Snapshot(
520      final Snapshot latest, final INodeDirectory snapshotCopy)
521          throws QuotaExceededException {
522    diffs.saveSelf2Snapshot(latest, this, snapshotCopy);
523    return this;
524  }
525
526  @Override
527  public INode saveChild2Snapshot(final INode child, final Snapshot latest,
528      final INode snapshotCopy, final INodeMap inodeMap)
529      throws QuotaExceededException {
530    Preconditions.checkArgument(!child.isDirectory(),
531        "child is a directory, child=%s", child);
532    if (latest == null) {
533      return child;
534    }
535
536    final DirectoryDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest, this);
537    if (diff.getChild(child.getLocalNameBytes(), false, this) != null) {
538      // it was already saved in the latest snapshot earlier.  
539      return child;
540    }
541
542    diff.diff.modify(snapshotCopy, child);
543    return child;
544  }
545
546  @Override
547  public boolean addChild(INode inode, boolean setModTime, Snapshot latest,
548      final INodeMap inodeMap) throws QuotaExceededException {
549    ChildrenDiff diff = null;
550    Integer undoInfo = null;
551    if (isInLatestSnapshot(latest)) {
552      diff = diffs.checkAndAddLatestSnapshotDiff(latest, this).diff;
553      undoInfo = diff.create(inode);
554    }
555    final boolean added = super.addChild(inode, setModTime, null, inodeMap);
556    if (!added && undoInfo != null) {
557      diff.undoCreate(inode, undoInfo);
558    }
559    return added; 
560  }
561
562  @Override
563  public boolean removeChild(INode child, Snapshot latest,
564      final INodeMap inodeMap) throws QuotaExceededException {
565    ChildrenDiff diff = null;
566    UndoInfo<INode> undoInfo = null;
567    // For a directory that is not a renamed node, if isInLatestSnapshot returns
568    // false, the directory is not in the latest snapshot, thus we do not need
569    // to record the removed child in any snapshot.
570    // For a directory that was moved/renamed, note that if the directory is in
571    // any of the previous snapshots, we will create a reference node for the 
572    // directory while rename, and isInLatestSnapshot will return true in that
573    // scenario (if all previous snapshots have been deleted, isInLatestSnapshot
574    // still returns false). Thus if isInLatestSnapshot returns false, the 
575    // directory node cannot be in any snapshot (not in current tree, nor in 
576    // previous src tree). Thus we do not need to record the removed child in 
577    // any snapshot.
578    if (isInLatestSnapshot(latest)) {
579      diff = diffs.checkAndAddLatestSnapshotDiff(latest, this).diff;
580      undoInfo = diff.delete(child);
581    }
582    final boolean removed = removeChild(child);
583    if (undoInfo != null) {
584      if (!removed) {
585        //remove failed, undo
586        diff.undoDelete(child, undoInfo);
587      }
588    }
589    return removed;
590  }
591  
592  @Override
593  public void replaceChildFileInSnapshot(final INodeFile oldChild,
594      final INodeFile newChild) {
595    super.replaceChildFileInSnapshot(oldChild, newChild);
596    diffs.replaceChild(ListType.DELETED, oldChild, newChild);
597    diffs.replaceChild(ListType.CREATED, oldChild, newChild);
598  }
599  
600  @Override
601  public void replaceChild(final INode oldChild, final INode newChild,
602      final INodeMap inodeMap) {
603    super.replaceChild(oldChild, newChild, inodeMap);
604    if (oldChild.getParentReference() != null && !newChild.isReference()) {
605      // oldChild is referred by a Reference node. Thus we are replacing the 
606      // referred inode, e.g., 
607      // INodeFileWithSnapshot -> INodeFileUnderConstructionWithSnapshot
608      // in this case, we do not need to update the diff list
609      return;
610    } else {
611      diffs.replaceChild(ListType.CREATED, oldChild, newChild);
612    }
613  }
614  
615  /**
616   * This method is usually called by the undo section of rename.
617   * 
618   * Before calling this function, in the rename operation, we replace the
619   * original src node (of the rename operation) with a reference node (WithName
620   * instance) in both the children list and a created list, delete the
621   * reference node from the children list, and add it to the corresponding
622   * deleted list.
623   * 
624   * To undo the above operations, we have the following steps in particular:
625   * 
626   * <pre>
627   * 1) remove the WithName node from the deleted list (if it exists) 
628   * 2) replace the WithName node in the created list with srcChild 
629   * 3) add srcChild back as a child of srcParent. Note that we already add 
630   * the node into the created list of a snapshot diff in step 2, we do not need
631   * to add srcChild to the created list of the latest snapshot.
632   * </pre>
633   * 
634   * We do not need to update quota usage because the old child is in the 
635   * deleted list before. 
636   * 
637   * @param oldChild
638   *          The reference node to be removed/replaced
639   * @param newChild
640   *          The node to be added back
641   * @param latestSnapshot
642   *          The latest snapshot. Note this may not be the last snapshot in the
643   *          {@link #diffs}, since the src tree of the current rename operation
644   *          may be the dst tree of a previous rename.
645   * @throws QuotaExceededException should not throw this exception
646   */
647  public void undoRename4ScrParent(final INodeReference oldChild,
648      final INode newChild, Snapshot latestSnapshot)
649      throws QuotaExceededException {
650    diffs.removeChild(ListType.DELETED, oldChild);
651    diffs.replaceChild(ListType.CREATED, oldChild, newChild);
652    // pass null for inodeMap since the parent node will not get replaced when
653    // undoing rename
654    addChild(newChild, true, null, null);
655  }
656  
657  /**
658   * Undo the rename operation for the dst tree, i.e., if the rename operation
659   * (with OVERWRITE option) removes a file/dir from the dst tree, add it back
660   * and delete possible record in the deleted list.  
661   */
662  public void undoRename4DstParent(final INode deletedChild,
663      Snapshot latestSnapshot) throws QuotaExceededException {
664    boolean removeDeletedChild = diffs.removeChild(ListType.DELETED,
665        deletedChild);
666    // pass null for inodeMap since the parent node will not get replaced when
667    // undoing rename
668    final boolean added = addChild(deletedChild, true, removeDeletedChild ? null
669        : latestSnapshot, null);
670    // update quota usage if adding is successfully and the old child has not
671    // been stored in deleted list before
672    if (added && !removeDeletedChild) {
673      final Quota.Counts counts = deletedChild.computeQuotaUsage();
674      addSpaceConsumed(counts.get(Quota.NAMESPACE),
675          counts.get(Quota.DISKSPACE), false);
676    }
677  }
678
679  @Override
680  public ReadOnlyList<INode> getChildrenList(Snapshot snapshot) {
681    final DirectoryDiff diff = diffs.getDiff(snapshot);
682    return diff != null? diff.getChildrenList(this): super.getChildrenList(null);
683  }
684
685  @Override
686  public INode getChild(byte[] name, Snapshot snapshot) {
687    final DirectoryDiff diff = diffs.getDiff(snapshot);
688    return diff != null? diff.getChild(name, true, this): super.getChild(name, null);
689  }
690
691  @Override
692  public String toDetailString() {
693    return super.toDetailString() + ", " + diffs;
694  }
695  
696  /**
697   * Get all the directories that are stored in some snapshot but not in the
698   * current children list. These directories are equivalent to the directories
699   * stored in the deletes lists.
700   */
701  public void getSnapshotDirectory(List<INodeDirectory> snapshotDir) {
702    for (DirectoryDiff sdiff : diffs) {
703      sdiff.getChildrenDiff().getDirsInDeleted(snapshotDir);
704    }
705  }
706
707  @Override
708  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
709      final BlocksMapUpdateInfo collectedBlocks,
710      final List<INode> removedINodes, final boolean countDiffChange)
711      throws QuotaExceededException {
712    Quota.Counts counts = Quota.Counts.newInstance();
713    Map<INode, INode> priorCreated = null;
714    Map<INode, INode> priorDeleted = null;
715    if (snapshot == null) { // delete the current directory
716      recordModification(prior, null);
717      // delete everything in created list
718      DirectoryDiff lastDiff = diffs.getLast();
719      if (lastDiff != null) {
720        counts.add(lastDiff.diff.destroyCreatedList(this, collectedBlocks,
721            removedINodes));
722      }
723    } else {
724      // update prior
725      prior = getDiffs().updatePrior(snapshot, prior);
726      // if there is a snapshot diff associated with prior, we need to record
727      // its original created and deleted list before deleting post
728      if (prior != null) {
729        DirectoryDiff priorDiff = this.getDiffs().getDiff(prior);
730        if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
731          List<INode> cList = priorDiff.diff.getList(ListType.CREATED);
732          List<INode> dList = priorDiff.diff.getList(ListType.DELETED);
733          priorCreated = cloneDiffList(cList);
734          priorDeleted = cloneDiffList(dList);
735        }
736      }
737      
738      counts.add(getDiffs().deleteSnapshotDiff(snapshot, prior, this, 
739          collectedBlocks, removedINodes, countDiffChange));
740      
741      // check priorDiff again since it may be created during the diff deletion
742      if (prior != null) {
743        DirectoryDiff priorDiff = this.getDiffs().getDiff(prior);
744        if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
745          // For files/directories created between "prior" and "snapshot", 
746          // we need to clear snapshot copies for "snapshot". Note that we must
747          // use null as prior in the cleanSubtree call. Files/directories that
748          // were created before "prior" will be covered by the later 
749          // cleanSubtreeRecursively call.
750          if (priorCreated != null) {
751            // we only check the node originally in prior's created list
752            for (INode cNode : priorDiff.getChildrenDiff().getList(
753                ListType.CREATED)) {
754              if (priorCreated.containsKey(cNode)) {
755                counts.add(cNode.cleanSubtree(snapshot, null, collectedBlocks,
756                    removedINodes, countDiffChange));
757              }
758            }
759          }
760          
761          // When a directory is moved from the deleted list of the posterior
762          // diff to the deleted list of this diff, we need to destroy its
763          // descendants that were 1) created after taking this diff and 2)
764          // deleted after taking posterior diff.
765
766          // For files moved from posterior's deleted list, we also need to
767          // delete its snapshot copy associated with the posterior snapshot.
768          
769          for (INode dNode : priorDiff.getChildrenDiff().getList(
770              ListType.DELETED)) {
771            if (priorDeleted == null || !priorDeleted.containsKey(dNode)) {
772              counts.add(cleanDeletedINode(dNode, snapshot, prior,
773                  collectedBlocks, removedINodes, countDiffChange));
774            }
775          }
776        }
777      }
778    }
779    counts.add(cleanSubtreeRecursively(snapshot, prior, collectedBlocks,
780        removedINodes, priorDeleted, countDiffChange));
781    
782    if (isQuotaSet()) {
783      this.addSpaceConsumed2Cache(-counts.get(Quota.NAMESPACE),
784          -counts.get(Quota.DISKSPACE));
785    }
786    return counts;
787  }
788  
789  /**
790   * Clean an inode while we move it from the deleted list of post to the
791   * deleted list of prior.
792   * @param inode The inode to clean.
793   * @param post The post snapshot.
794   * @param prior The prior snapshot.
795   * @param collectedBlocks Used to collect blocks for later deletion.
796   * @return Quota usage update.
797   */
798  private static Quota.Counts cleanDeletedINode(INode inode,
799      final Snapshot post, final Snapshot prior,
800      final BlocksMapUpdateInfo collectedBlocks,
801      final List<INode> removedINodes, final boolean countDiffChange) 
802      throws QuotaExceededException {
803    Quota.Counts counts = Quota.Counts.newInstance();
804    Deque<INode> queue = new ArrayDeque<INode>();
805    queue.addLast(inode);
806    while (!queue.isEmpty()) {
807      INode topNode = queue.pollFirst();
808      if (topNode instanceof INodeReference.WithName) {
809        INodeReference.WithName wn = (INodeReference.WithName) topNode;
810        if (wn.getLastSnapshotId() >= post.getId()) {
811          wn.cleanSubtree(post, prior, collectedBlocks, removedINodes,
812              countDiffChange);
813        }
814        // For DstReference node, since the node is not in the created list of
815        // prior, we should treat it as regular file/dir
816      } else if (topNode.isFile()
817          && topNode.asFile() instanceof FileWithSnapshot) {
818        FileWithSnapshot fs = (FileWithSnapshot) topNode.asFile();
819        counts.add(fs.getDiffs().deleteSnapshotDiff(post, prior,
820            topNode.asFile(), collectedBlocks, removedINodes, countDiffChange));
821      } else if (topNode.isDirectory()) {
822        INodeDirectory dir = topNode.asDirectory();
823        ChildrenDiff priorChildrenDiff = null;
824        if (dir instanceof INodeDirectoryWithSnapshot) {
825          // delete files/dirs created after prior. Note that these
826          // files/dirs, along with inode, were deleted right after post.
827          INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) dir;
828          DirectoryDiff priorDiff = sdir.getDiffs().getDiff(prior);
829          if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
830            priorChildrenDiff = priorDiff.getChildrenDiff();
831            counts.add(priorChildrenDiff.destroyCreatedList(sdir,
832                collectedBlocks, removedINodes));
833          }
834        }
835        
836        for (INode child : dir.getChildrenList(prior)) {
837          if (priorChildrenDiff != null
838              && priorChildrenDiff.search(ListType.DELETED,
839                  child.getLocalNameBytes()) != null) {
840            continue;
841          }
842          queue.addLast(child);
843        }
844      }
845    }
846    return counts;
847  }
848
849  @Override
850  public void destroyAndCollectBlocks(
851      final BlocksMapUpdateInfo collectedBlocks, 
852      final List<INode> removedINodes) {
853    // destroy its diff list
854    for (DirectoryDiff diff : diffs) {
855      diff.destroyDiffAndCollectBlocks(this, collectedBlocks, removedINodes);
856    }
857    diffs.clear();
858    super.destroyAndCollectBlocks(collectedBlocks, removedINodes);
859  }
860
861  @Override
862  public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
863      boolean useCache, int lastSnapshotId) {
864    if ((useCache && isQuotaSet()) || lastSnapshotId == Snapshot.INVALID_ID) {
865      return super.computeQuotaUsage(counts, useCache, lastSnapshotId);
866    }
867    
868    Snapshot lastSnapshot = diffs.getSnapshotById(lastSnapshotId);
869    
870    ReadOnlyList<INode> childrenList = getChildrenList(lastSnapshot);
871    for (INode child : childrenList) {
872      child.computeQuotaUsage(counts, useCache, lastSnapshotId);
873    }
874    
875    counts.add(Quota.NAMESPACE, 1);
876    return counts;
877  }
878  
879  @Override
880  public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
881    super.computeQuotaUsage4CurrentDirectory(counts);
882    for(DirectoryDiff d : diffs) {
883      for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
884        deleted.computeQuotaUsage(counts, false, Snapshot.INVALID_ID);
885      }
886    }
887    counts.add(Quota.NAMESPACE, diffs.asList().size());
888    return counts;
889  }
890
891  @Override
892  public ContentSummaryComputationContext computeContentSummary(
893      final ContentSummaryComputationContext summary) {
894    // Snapshot summary calc won't be relinquishing locks in the middle.
895    // Do this first and handover to parent.
896    computeContentSummary4Snapshot(summary.getCounts());
897    super.computeContentSummary(summary);
898    return summary;
899  }
900
901  private void computeContentSummary4Snapshot(final Content.Counts counts) {
902    // Create a new blank summary context for blocking processing of subtree.
903    ContentSummaryComputationContext summary = 
904        new ContentSummaryComputationContext();
905    for(DirectoryDiff d : diffs) {
906      for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
907        deleted.computeContentSummary(summary);
908      }
909    }
910    // Add the counts from deleted trees.
911    counts.add(summary.getCounts());
912    // Add the deleted directory count.
913    counts.add(Content.DIRECTORY, diffs.asList().size());
914  }
915  
916  private static Map<INode, INode> cloneDiffList(List<INode> diffList) {
917    if (diffList == null || diffList.size() == 0) {
918      return null;
919    }
920    Map<INode, INode> map = new HashMap<INode, INode>(diffList.size());
921    for (INode node : diffList) {
922      map.put(node, node);
923    }
924    return map;
925  }
926  
927  /**
928   * Destroy a subtree under a DstReference node.
929   */
930  public static void destroyDstSubtree(INode inode, final Snapshot snapshot,
931      final Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,
932      final List<INode> removedINodes) throws QuotaExceededException {
933    Preconditions.checkArgument(prior != null);
934    if (inode.isReference()) {
935      if (inode instanceof INodeReference.WithName && snapshot != null) {
936        // this inode has been renamed before the deletion of the DstReference
937        // subtree
938        inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes,
939            true);
940      } else { 
941        // for DstReference node, continue this process to its subtree
942        destroyDstSubtree(inode.asReference().getReferredINode(), snapshot,
943            prior, collectedBlocks, removedINodes);
944      }
945    } else if (inode.isFile()) {
946      inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes, true);
947    } else if (inode.isDirectory()) {
948      Map<INode, INode> excludedNodes = null;
949      if (inode instanceof INodeDirectoryWithSnapshot) {
950        INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) inode;
951        
952        DirectoryDiffList diffList = sdir.getDiffs();
953        DirectoryDiff priorDiff = diffList.getDiff(prior);
954        if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
955          List<INode> dList = priorDiff.diff.getList(ListType.DELETED);
956          excludedNodes = cloneDiffList(dList);
957        }
958        
959        if (snapshot != null) {
960          diffList.deleteSnapshotDiff(snapshot, prior, sdir, collectedBlocks,
961              removedINodes, true);
962        }
963        priorDiff = diffList.getDiff(prior);
964        if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
965          priorDiff.diff.destroyCreatedList(sdir, collectedBlocks,
966              removedINodes);
967        }
968      }
969      for (INode child : inode.asDirectory().getChildrenList(prior)) {
970        if (excludedNodes != null && excludedNodes.containsKey(child)) {
971          continue;
972        }
973        destroyDstSubtree(child, snapshot, prior, collectedBlocks,
974            removedINodes);
975      }
976    }
977  }
978}