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.IOException;
021import java.io.PrintWriter;
022import java.util.ArrayList;
023import java.util.Collections;
024import java.util.Comparator;
025import java.util.HashMap;
026import java.util.Iterator;
027import java.util.List;
028import java.util.Map;
029import java.util.SortedMap;
030import java.util.TreeMap;
031
032import org.apache.hadoop.HadoopIllegalArgumentException;
033import org.apache.hadoop.classification.InterfaceAudience;
034import org.apache.hadoop.hdfs.DFSUtil;
035import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
036import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
037import org.apache.hadoop.hdfs.protocol.SnapshotException;
038import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
039import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
040import org.apache.hadoop.hdfs.server.namenode.Content;
041import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
042import org.apache.hadoop.hdfs.server.namenode.INode;
043import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
044import org.apache.hadoop.hdfs.server.namenode.INodeFile;
045import org.apache.hadoop.hdfs.server.namenode.INodeMap;
046import org.apache.hadoop.hdfs.server.namenode.Quota;
047import org.apache.hadoop.hdfs.util.Diff.ListType;
048import org.apache.hadoop.hdfs.util.ReadOnlyList;
049import org.apache.hadoop.util.Time;
050
051import com.google.common.base.Preconditions;
052import com.google.common.primitives.SignedBytes;
053
054/**
055 * Directories where taking snapshots is allowed.
056 * 
057 * Like other {@link INode} subclasses, this class is synchronized externally
058 * by the namesystem and FSDirectory locks.
059 */
060@InterfaceAudience.Private
061public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
062  /** Limit the number of snapshot per snapshottable directory. */
063  static final int SNAPSHOT_LIMIT = 1 << 16;
064
065  /** Cast INode to INodeDirectorySnapshottable. */
066  static public INodeDirectorySnapshottable valueOf(
067      INode inode, String src) throws IOException {
068    final INodeDirectory dir = INodeDirectory.valueOf(inode, src);
069    if (!dir.isSnapshottable()) {
070      throw new SnapshotException(
071          "Directory is not a snapshottable directory: " + src);
072    }
073    return (INodeDirectorySnapshottable)dir;
074  }
075  
076  /**
077   * A class describing the difference between snapshots of a snapshottable
078   * directory.
079   */
080  public static class SnapshotDiffInfo {
081    /** Compare two inodes based on their full names */
082    public static final Comparator<INode> INODE_COMPARATOR = 
083        new Comparator<INode>() {
084      @Override
085      public int compare(INode left, INode right) {
086        if (left == null) {
087          return right == null ? 0 : -1;
088        } else {
089          if (right == null) {
090            return 1;
091          } else {
092            int cmp = compare(left.getParent(), right.getParent());
093            return cmp == 0 ? SignedBytes.lexicographicalComparator().compare(
094                left.getLocalNameBytes(), right.getLocalNameBytes()) : cmp;
095          }
096        }
097      }
098    };
099    
100    /** The root directory of the snapshots */
101    private final INodeDirectorySnapshottable snapshotRoot;
102    /** The starting point of the difference */
103    private final Snapshot from;
104    /** The end point of the difference */
105    private final Snapshot to;
106    /**
107     * A map recording modified INodeFile and INodeDirectory and their relative
108     * path corresponding to the snapshot root. Sorted based on their names.
109     */ 
110    private final SortedMap<INode, byte[][]> diffMap = 
111        new TreeMap<INode, byte[][]>(INODE_COMPARATOR);
112    /**
113     * A map capturing the detailed difference about file creation/deletion.
114     * Each key indicates a directory whose children have been changed between
115     * the two snapshots, while its associated value is a {@link ChildrenDiff}
116     * storing the changes (creation/deletion) happened to the children (files).
117     */
118    private final Map<INodeDirectoryWithSnapshot, ChildrenDiff> dirDiffMap = 
119        new HashMap<INodeDirectoryWithSnapshot, ChildrenDiff>();
120    
121    SnapshotDiffInfo(INodeDirectorySnapshottable snapshotRoot, Snapshot start,
122        Snapshot end) {
123      this.snapshotRoot = snapshotRoot;
124      this.from = start;
125      this.to = end;
126    }
127    
128    /** Add a dir-diff pair */
129    private void addDirDiff(INodeDirectoryWithSnapshot dir,
130        byte[][] relativePath, ChildrenDiff diff) {
131      dirDiffMap.put(dir, diff);
132      diffMap.put(dir, relativePath);
133    }
134    
135    /** Add a modified file */ 
136    private void addFileDiff(INodeFile file, byte[][] relativePath) {
137      diffMap.put(file, relativePath);
138    }
139    
140    /** @return True if {@link #from} is earlier than {@link #to} */
141    private boolean isFromEarlier() {
142      return Snapshot.ID_COMPARATOR.compare(from, to) < 0;
143    }
144    
145    /**
146     * Generate a {@link SnapshotDiffReport} based on detailed diff information.
147     * @return A {@link SnapshotDiffReport} describing the difference
148     */
149    public SnapshotDiffReport generateReport() {
150      List<DiffReportEntry> diffReportList = new ArrayList<DiffReportEntry>();
151      for (INode node : diffMap.keySet()) {
152        diffReportList.add(new DiffReportEntry(DiffType.MODIFY, diffMap
153            .get(node)));
154        if (node.isDirectory()) {
155          ChildrenDiff dirDiff = dirDiffMap.get(node);
156          List<DiffReportEntry> subList = dirDiff.generateReport(
157              diffMap.get(node), (INodeDirectoryWithSnapshot) node,
158              isFromEarlier());
159          diffReportList.addAll(subList);
160        }
161      }
162      return new SnapshotDiffReport(snapshotRoot.getFullPathName(),
163          Snapshot.getSnapshotName(from), Snapshot.getSnapshotName(to),
164          diffReportList);
165    }
166  }
167
168  /**
169   * Snapshots of this directory in ascending order of snapshot names.
170   * Note that snapshots in ascending order of snapshot id are stored in
171   * {@link INodeDirectoryWithSnapshot}.diffs (a private field).
172   */
173  private final List<Snapshot> snapshotsByNames = new ArrayList<Snapshot>();
174
175  /**
176   * @return {@link #snapshotsByNames}
177   */
178  ReadOnlyList<Snapshot> getSnapshotsByNames() {
179    return ReadOnlyList.Util.asReadOnlyList(this.snapshotsByNames);
180  }
181  
182  /** Number of snapshots allowed. */
183  private int snapshotQuota = SNAPSHOT_LIMIT;
184
185  public INodeDirectorySnapshottable(INodeDirectory dir) {
186    super(dir, true, dir instanceof INodeDirectoryWithSnapshot ? 
187        ((INodeDirectoryWithSnapshot) dir).getDiffs(): null);
188  }
189  
190  /** @return the number of existing snapshots. */
191  public int getNumSnapshots() {
192    return snapshotsByNames.size();
193  }
194  
195  private int searchSnapshot(byte[] snapshotName) {
196    return Collections.binarySearch(snapshotsByNames, snapshotName);
197  }
198
199  /** @return the snapshot with the given name. */
200  public Snapshot getSnapshot(byte[] snapshotName) {
201    final int i = searchSnapshot(snapshotName);
202    return i < 0? null: snapshotsByNames.get(i);
203  }
204  
205  /** @return {@link #snapshotsByNames} as a {@link ReadOnlyList} */
206  public ReadOnlyList<Snapshot> getSnapshotList() {
207    return ReadOnlyList.Util.asReadOnlyList(snapshotsByNames);
208  }
209  
210  /**
211   * Rename a snapshot
212   * @param path
213   *          The directory path where the snapshot was taken. Used for
214   *          generating exception message.
215   * @param oldName
216   *          Old name of the snapshot
217   * @param newName
218   *          New name the snapshot will be renamed to
219   * @throws SnapshotException
220   *           Throw SnapshotException when either the snapshot with the old
221   *           name does not exist or a snapshot with the new name already
222   *           exists
223   */
224  public void renameSnapshot(String path, String oldName, String newName)
225      throws SnapshotException {
226    if (newName.equals(oldName)) {
227      return;
228    }
229    final int indexOfOld = searchSnapshot(DFSUtil.string2Bytes(oldName));
230    if (indexOfOld < 0) {
231      throw new SnapshotException("The snapshot " + oldName
232          + " does not exist for directory " + path);
233    } else {
234      final byte[] newNameBytes = DFSUtil.string2Bytes(newName);
235      int indexOfNew = searchSnapshot(newNameBytes);
236      if (indexOfNew > 0) {
237        throw new SnapshotException("The snapshot " + newName
238            + " already exists for directory " + path);
239      }
240      // remove the one with old name from snapshotsByNames
241      Snapshot snapshot = snapshotsByNames.remove(indexOfOld);
242      final INodeDirectory ssRoot = snapshot.getRoot();
243      ssRoot.setLocalName(newNameBytes);
244      indexOfNew = -indexOfNew - 1;
245      if (indexOfNew <= indexOfOld) {
246        snapshotsByNames.add(indexOfNew, snapshot);
247      } else { // indexOfNew > indexOfOld
248        snapshotsByNames.add(indexOfNew - 1, snapshot);
249      }
250    }
251  }
252
253  public int getSnapshotQuota() {
254    return snapshotQuota;
255  }
256
257  public void setSnapshotQuota(int snapshotQuota) {
258    if (snapshotQuota < 0) {
259      throw new HadoopIllegalArgumentException(
260          "Cannot set snapshot quota to " + snapshotQuota + " < 0");
261    }
262    this.snapshotQuota = snapshotQuota;
263  }
264
265  @Override
266  public boolean isSnapshottable() {
267    return true;
268  }
269  
270  /**
271   * Simply add a snapshot into the {@link #snapshotsByNames}. Used by FSImage
272   * loading.
273   */
274  void addSnapshot(Snapshot snapshot) {
275    this.snapshotsByNames.add(snapshot);
276  }
277
278  /** Add a snapshot. */
279  Snapshot addSnapshot(int id, String name) throws SnapshotException,
280      QuotaExceededException {
281    //check snapshot quota
282    final int n = getNumSnapshots();
283    if (n + 1 > snapshotQuota) {
284      throw new SnapshotException("Failed to add snapshot: there are already "
285          + n + " snapshot(s) and the snapshot quota is "
286          + snapshotQuota);
287    }
288    final Snapshot s = new Snapshot(id, name, this);
289    final byte[] nameBytes = s.getRoot().getLocalNameBytes();
290    final int i = searchSnapshot(nameBytes);
291    if (i >= 0) {
292      throw new SnapshotException("Failed to add snapshot: there is already a "
293          + "snapshot with the same name \"" + Snapshot.getSnapshotName(s) + "\".");
294    }
295
296    final DirectoryDiff d = getDiffs().addDiff(s, this);
297    d.snapshotINode = s.getRoot();
298    snapshotsByNames.add(-i - 1, s);
299
300    //set modification time
301    updateModificationTime(Time.now(), null, null);
302    s.getRoot().setModificationTime(getModificationTime(), null, null);
303    return s;
304  }
305  
306  /**
307   * Remove the snapshot with the given name from {@link #snapshotsByNames},
308   * and delete all the corresponding DirectoryDiff.
309   * 
310   * @param snapshotName The name of the snapshot to be removed
311   * @param collectedBlocks Used to collect information to update blocksMap
312   * @return The removed snapshot. Null if no snapshot with the given name 
313   *         exists.
314   */
315  Snapshot removeSnapshot(String snapshotName,
316      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
317      throws SnapshotException {
318    final int i = searchSnapshot(DFSUtil.string2Bytes(snapshotName));
319    if (i < 0) {
320      throw new SnapshotException("Cannot delete snapshot " + snapshotName
321          + " from path " + this.getFullPathName()
322          + ": the snapshot does not exist.");
323    } else {
324      final Snapshot snapshot = snapshotsByNames.get(i);
325      Snapshot prior = Snapshot.findLatestSnapshot(this, snapshot);
326      try {
327        Quota.Counts counts = cleanSubtree(snapshot, prior, collectedBlocks,
328            removedINodes, true);
329        INodeDirectory parent = getParent();
330        if (parent != null) {
331          // there will not be any WithName node corresponding to the deleted 
332          // snapshot, thus only update the quota usage in the current tree
333          parent.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
334              -counts.get(Quota.DISKSPACE), true);
335        }
336      } catch(QuotaExceededException e) {
337        LOG.error("BUG: removeSnapshot increases namespace usage.", e);
338      }
339      // remove from snapshotsByNames after successfully cleaning the subtree
340      snapshotsByNames.remove(i);
341      return snapshot;
342    }
343  }
344  
345  @Override
346  public ContentSummaryComputationContext computeContentSummary(
347      final ContentSummaryComputationContext summary) {
348    super.computeContentSummary(summary);
349    summary.getCounts().add(Content.SNAPSHOT, snapshotsByNames.size());
350    summary.getCounts().add(Content.SNAPSHOTTABLE_DIRECTORY, 1);
351    return summary;
352  }
353
354  /**
355   * Compute the difference between two snapshots (or a snapshot and the current
356   * directory) of the directory.
357   * 
358   * @param from The name of the start point of the comparison. Null indicating
359   *          the current tree.
360   * @param to The name of the end point. Null indicating the current tree.
361   * @return The difference between the start/end points.
362   * @throws SnapshotException If there is no snapshot matching the starting
363   *           point, or if endSnapshotName is not null but cannot be identified
364   *           as a previous snapshot.
365   */
366  SnapshotDiffInfo computeDiff(final String from, final String to)
367      throws SnapshotException {
368    Snapshot fromSnapshot = getSnapshotByName(from);
369    Snapshot toSnapshot = getSnapshotByName(to);
370    // if the start point is equal to the end point, return null
371    if (from.equals(to)) {
372      return null;
373    }
374    SnapshotDiffInfo diffs = new SnapshotDiffInfo(this, fromSnapshot,
375        toSnapshot);
376    computeDiffRecursively(this, new ArrayList<byte[]>(), diffs);
377    return diffs;
378  }
379  
380  /**
381   * Find the snapshot matching the given name.
382   * 
383   * @param snapshotName The name of the snapshot.
384   * @return The corresponding snapshot. Null if snapshotName is null or empty.
385   * @throws SnapshotException If snapshotName is not null or empty, but there
386   *           is no snapshot matching the name.
387   */
388  private Snapshot getSnapshotByName(String snapshotName)
389      throws SnapshotException {
390    Snapshot s = null;
391    if (snapshotName != null && !snapshotName.isEmpty()) {
392      final int index = searchSnapshot(DFSUtil.string2Bytes(snapshotName));
393      if (index < 0) {
394        throw new SnapshotException("Cannot find the snapshot of directory "
395            + this.getFullPathName() + " with name " + snapshotName);
396      }
397      s = snapshotsByNames.get(index);
398    }
399    return s;
400  }
401  
402  /**
403   * Recursively compute the difference between snapshots under a given
404   * directory/file.
405   * @param node The directory/file under which the diff is computed. 
406   * @param parentPath Relative path (corresponding to the snapshot root) of 
407   *                   the node's parent.
408   * @param diffReport data structure used to store the diff.
409   */
410  private void computeDiffRecursively(INode node, List<byte[]> parentPath,
411      SnapshotDiffInfo diffReport) {
412    ChildrenDiff diff = new ChildrenDiff();
413    byte[][] relativePath = parentPath.toArray(new byte[parentPath.size()][]);
414    if (node.isDirectory()) {
415      INodeDirectory dir = node.asDirectory();
416      if (dir instanceof INodeDirectoryWithSnapshot) {
417        INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) dir;
418        boolean change = sdir.computeDiffBetweenSnapshots(
419            diffReport.from, diffReport.to, diff);
420        if (change) {
421          diffReport.addDirDiff(sdir, relativePath, diff);
422        }
423      }
424      ReadOnlyList<INode> children = dir.getChildrenList(diffReport
425          .isFromEarlier() ? diffReport.to : diffReport.from);
426      for (INode child : children) {
427        final byte[] name = child.getLocalNameBytes();
428        if (diff.searchIndex(ListType.CREATED, name) < 0
429            && diff.searchIndex(ListType.DELETED, name) < 0) {
430          parentPath.add(name);
431          computeDiffRecursively(child, parentPath, diffReport);
432          parentPath.remove(parentPath.size() - 1);
433        }
434      }
435    } else if (node.isFile() && node.asFile() instanceof FileWithSnapshot) {
436      FileWithSnapshot file = (FileWithSnapshot) node.asFile();
437      Snapshot earlierSnapshot = diffReport.isFromEarlier() ? diffReport.from
438          : diffReport.to;
439      Snapshot laterSnapshot = diffReport.isFromEarlier() ? diffReport.to
440          : diffReport.from;
441      boolean change = file.getDiffs().changedBetweenSnapshots(earlierSnapshot,
442          laterSnapshot);
443      if (change) {
444        diffReport.addFileDiff(file.asINodeFile(), relativePath);
445      }
446    }
447  }
448  
449  /**
450   * Replace itself with {@link INodeDirectoryWithSnapshot} or
451   * {@link INodeDirectory} depending on the latest snapshot.
452   */
453  INodeDirectory replaceSelf(final Snapshot latest, final INodeMap inodeMap)
454      throws QuotaExceededException {
455    if (latest == null) {
456      Preconditions.checkState(getLastSnapshot() == null,
457          "latest == null but getLastSnapshot() != null, this=%s", this);
458      return replaceSelf4INodeDirectory(inodeMap);
459    } else {
460      return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
461          .recordModification(latest, null);
462    }
463  }
464
465  @Override
466  public String toDetailString() {
467    return super.toDetailString() + ", snapshotsByNames=" + snapshotsByNames;
468  }
469
470  @Override
471  public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
472      Snapshot snapshot) {
473    super.dumpTreeRecursively(out, prefix, snapshot);
474
475    if (snapshot == null) {
476      out.println();
477      out.print(prefix);
478
479      out.print("Snapshot of ");
480      final String name = getLocalName();
481      out.print(name.isEmpty()? "/": name);
482      out.print(": quota=");
483      out.print(getSnapshotQuota());
484
485      int n = 0;
486      for(DirectoryDiff diff : getDiffs()) {
487        if (diff.isSnapshotRoot()) {
488          n++;
489        }
490      }
491      Preconditions.checkState(n == snapshotsByNames.size());
492      out.print(", #snapshot=");
493      out.println(n);
494
495      dumpTreeRecursively(out, prefix, new Iterable<SnapshotAndINode>() {
496        @Override
497        public Iterator<SnapshotAndINode> iterator() {
498          return new Iterator<SnapshotAndINode>() {
499            final Iterator<DirectoryDiff> i = getDiffs().iterator();
500            private DirectoryDiff next = findNext();
501  
502            private DirectoryDiff findNext() {
503              for(; i.hasNext(); ) {
504                final DirectoryDiff diff = i.next();
505                if (diff.isSnapshotRoot()) {
506                  return diff;
507                }
508              }
509              return null;
510            }
511
512            @Override
513            public boolean hasNext() {
514              return next != null;
515            }
516  
517            @Override
518            public SnapshotAndINode next() {
519              final Snapshot s = next.snapshot;
520              final SnapshotAndINode pair = new SnapshotAndINode(s);
521              next = findNext();
522              return pair;
523            }
524  
525            @Override
526            public void remove() {
527              throw new UnsupportedOperationException();
528            }
529          };
530        }
531      });
532    }
533  }
534}