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.DataInput;
021import java.io.DataOutput;
022import java.io.IOException;
023import java.util.ArrayList;
024import java.util.Collections;
025import java.util.HashMap;
026import java.util.List;
027import java.util.Map;
028import java.util.concurrent.atomic.AtomicInteger;
029
030import org.apache.hadoop.hdfs.DFSUtil;
031import org.apache.hadoop.hdfs.protocol.SnapshotException;
032import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
033import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
034import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
035import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
036import org.apache.hadoop.hdfs.server.namenode.INode;
037import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
038import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
039import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
040import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable.SnapshotDiffInfo;
041
042/**
043 * Manage snapshottable directories and their snapshots.
044 * 
045 * This class includes operations that create, access, modify snapshots and/or
046 * snapshot-related data. In general, the locking structure of snapshot
047 * operations is: <br>
048 * 
049 * 1. Lock the {@link FSNamesystem} lock in {@link FSNamesystem} before calling
050 * into {@link SnapshotManager} methods.<br>
051 * 2. Lock the {@link FSDirectory} lock for the {@link SnapshotManager} methods
052 * if necessary.
053 */
054public class SnapshotManager implements SnapshotStats {
055  private boolean allowNestedSnapshots = false;
056  private final FSDirectory fsdir;
057  private static final int SNAPSHOT_ID_BIT_WIDTH = 24;
058
059  private final AtomicInteger numSnapshots = new AtomicInteger();
060
061  private int snapshotCounter = 0;
062  
063  /** All snapshottable directories in the namesystem. */
064  private final Map<Long, INodeDirectorySnapshottable> snapshottables
065      = new HashMap<Long, INodeDirectorySnapshottable>();
066
067  public SnapshotManager(final FSDirectory fsdir) {
068    this.fsdir = fsdir;
069  }
070
071  /** Used in tests only */
072  void setAllowNestedSnapshots(boolean allowNestedSnapshots) {
073    this.allowNestedSnapshots = allowNestedSnapshots;
074  }
075
076  private void checkNestedSnapshottable(INodeDirectory dir, String path)
077      throws SnapshotException {
078    if (allowNestedSnapshots) {
079      return;
080    }
081
082    for(INodeDirectorySnapshottable s : snapshottables.values()) {
083      if (s.isAncestorDirectory(dir)) {
084        throw new SnapshotException(
085            "Nested snapshottable directories not allowed: path=" + path
086            + ", the subdirectory " + s.getFullPathName()
087            + " is already a snapshottable directory.");
088      }
089      if (dir.isAncestorDirectory(s)) {
090        throw new SnapshotException(
091            "Nested snapshottable directories not allowed: path=" + path
092            + ", the ancestor " + s.getFullPathName()
093            + " is already a snapshottable directory.");
094      }
095    }
096  }
097
098  /**
099   * Set the given directory as a snapshottable directory.
100   * If the path is already a snapshottable directory, update the quota.
101   */
102  public void setSnapshottable(final String path, boolean checkNestedSnapshottable)
103      throws IOException {
104    final INodesInPath iip = fsdir.getINodesInPath4Write(path);
105    final INodeDirectory d = INodeDirectory.valueOf(iip.getLastINode(), path);
106    if (checkNestedSnapshottable) {
107      checkNestedSnapshottable(d, path);
108    }
109
110
111    final INodeDirectorySnapshottable s;
112    if (d.isSnapshottable()) {
113      //The directory is already a snapshottable directory.
114      s = (INodeDirectorySnapshottable)d; 
115      s.setSnapshotQuota(INodeDirectorySnapshottable.SNAPSHOT_LIMIT);
116    } else {
117      s = d.replaceSelf4INodeDirectorySnapshottable(iip.getLatestSnapshot(),
118          fsdir.getINodeMap());
119    }
120    addSnapshottable(s);
121  }
122  
123  /** Add the given snapshottable directory to {@link #snapshottables}. */
124  public void addSnapshottable(INodeDirectorySnapshottable dir) {
125    snapshottables.put(dir.getId(), dir);
126  }
127
128  /** Remove the given snapshottable directory from {@link #snapshottables}. */
129  private void removeSnapshottable(INodeDirectorySnapshottable s) {
130    snapshottables.remove(s.getId());
131  }
132  
133  /** Remove snapshottable directories from {@link #snapshottables} */
134  public void removeSnapshottable(List<INodeDirectorySnapshottable> toRemove) {
135    if (toRemove != null) {
136      for (INodeDirectorySnapshottable s : toRemove) {
137        removeSnapshottable(s);
138      }
139    }
140  }
141
142  /**
143   * Set the given snapshottable directory to non-snapshottable.
144   * 
145   * @throws SnapshotException if there are snapshots in the directory.
146   */
147  public void resetSnapshottable(final String path) throws IOException {
148    final INodesInPath iip = fsdir.getINodesInPath4Write(path);
149    final INodeDirectory d = INodeDirectory.valueOf(iip.getLastINode(), path);
150    if (!d.isSnapshottable()) {
151      // the directory is already non-snapshottable
152      return;
153    }
154    final INodeDirectorySnapshottable s = (INodeDirectorySnapshottable) d;
155    if (s.getNumSnapshots() > 0) {
156      throw new SnapshotException("The directory " + path + " has snapshot(s). "
157          + "Please redo the operation after removing all the snapshots.");
158    }
159
160    if (s == fsdir.getRoot()) {
161      s.setSnapshotQuota(0); 
162    } else {
163      s.replaceSelf(iip.getLatestSnapshot(), fsdir.getINodeMap());
164    }
165    removeSnapshottable(s);
166  }
167
168  /**
169  * Find the source root directory where the snapshot will be taken
170  * for a given path.
171  *
172  * @param path The directory path where the snapshot will be taken.
173  * @return Snapshottable directory.
174  * @throws IOException
175  *           Throw IOException when the given path does not lead to an
176  *           existing snapshottable directory.
177  */
178  public INodeDirectorySnapshottable getSnapshottableRoot(final String path
179      ) throws IOException {
180    final INodesInPath i = fsdir.getINodesInPath4Write(path);
181    return INodeDirectorySnapshottable.valueOf(i.getLastINode(), path);
182  }
183
184  /**
185   * Create a snapshot of the given path.
186   * It is assumed that the caller will perform synchronization.
187   *
188   * @param path
189   *          The directory path where the snapshot will be taken.
190   * @param snapshotName
191   *          The name of the snapshot.
192   * @throws IOException
193   *           Throw IOException when 1) the given path does not lead to an
194   *           existing snapshottable directory, and/or 2) there exists a
195   *           snapshot with the given name for the directory, and/or 3)
196   *           snapshot number exceeds quota
197   */
198  public String createSnapshot(final String path, String snapshotName
199      ) throws IOException {
200    INodeDirectorySnapshottable srcRoot = getSnapshottableRoot(path);
201
202    if (snapshotCounter == getMaxSnapshotID()) {
203      // We have reached the maximum allowable snapshot ID and since we don't
204      // handle rollover we will fail all subsequent snapshot creation
205      // requests.
206      //
207      throw new SnapshotException(
208          "Failed to create the snapshot. The FileSystem has run out of " +
209          "snapshot IDs and ID rollover is not supported.");
210    }
211
212    srcRoot.addSnapshot(snapshotCounter, snapshotName);
213      
214    //create success, update id
215    snapshotCounter++;
216    numSnapshots.getAndIncrement();
217    return Snapshot.getSnapshotPath(path, snapshotName);
218  }
219  
220  /**
221   * Delete a snapshot for a snapshottable directory
222   * @param path Path to the directory where the snapshot was taken
223   * @param snapshotName Name of the snapshot to be deleted
224   * @param collectedBlocks Used to collect information to update blocksMap 
225   * @throws IOException
226   */
227  public void deleteSnapshot(final String path, final String snapshotName,
228      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
229      throws IOException {
230    // parse the path, and check if the path is a snapshot path
231    // the INodeDirectorySnapshottable#valueOf method will throw Exception 
232    // if the path is not for a snapshottable directory
233    INodeDirectorySnapshottable srcRoot = getSnapshottableRoot(path);
234    srcRoot.removeSnapshot(snapshotName, collectedBlocks, removedINodes);
235    numSnapshots.getAndDecrement();
236  }
237
238  /**
239   * Rename the given snapshot
240   * @param path
241   *          The directory path where the snapshot was taken
242   * @param oldSnapshotName
243   *          Old name of the snapshot
244   * @param newSnapshotName
245   *          New name of the snapshot
246   * @throws IOException
247   *           Throw IOException when 1) the given path does not lead to an
248   *           existing snapshottable directory, and/or 2) the snapshot with the
249   *           old name does not exist for the directory, and/or 3) there exists
250   *           a snapshot with the new name for the directory
251   */
252  public void renameSnapshot(final String path, final String oldSnapshotName,
253      final String newSnapshotName) throws IOException {
254    // Find the source root directory path where the snapshot was taken.
255    // All the check for path has been included in the valueOf method.
256    final INodeDirectorySnapshottable srcRoot
257        = INodeDirectorySnapshottable.valueOf(fsdir.getINode(path), path);
258    // Note that renameSnapshot and createSnapshot are synchronized externally
259    // through FSNamesystem's write lock
260    srcRoot.renameSnapshot(path, oldSnapshotName, newSnapshotName);
261  }
262  
263  @Override
264  public int getNumSnapshottableDirs() {
265    return snapshottables.size();
266  }
267
268  @Override
269  public int getNumSnapshots() {
270    return numSnapshots.get();
271  }
272  
273  /**
274   * Write {@link #snapshotCounter}, {@link #numSnapshots},
275   * and all snapshots to the DataOutput.
276   */
277  public void write(DataOutput out) throws IOException {
278    out.writeInt(snapshotCounter);
279    out.writeInt(numSnapshots.get());
280
281    // write all snapshots.
282    for(INodeDirectorySnapshottable snapshottableDir : snapshottables.values()) {
283      for(Snapshot s : snapshottableDir.getSnapshotsByNames()) {
284        s.write(out);
285      }
286    }
287  }
288  
289  /**
290   * Read values of {@link #snapshotCounter}, {@link #numSnapshots}, and
291   * all snapshots from the DataInput
292   */
293  public Map<Integer, Snapshot> read(DataInput in, FSImageFormat.Loader loader
294      ) throws IOException {
295    snapshotCounter = in.readInt();
296    numSnapshots.set(in.readInt());
297    
298    // read snapshots
299    final Map<Integer, Snapshot> snapshotMap = new HashMap<Integer, Snapshot>();
300    for(int i = 0; i < numSnapshots.get(); i++) {
301      final Snapshot s = Snapshot.read(in, loader);
302      snapshotMap.put(s.getId(), s);
303    }
304    return snapshotMap;
305  }
306  
307  /**
308   * List all the snapshottable directories that are owned by the current user.
309   * @param userName Current user name.
310   * @return Snapshottable directories that are owned by the current user,
311   *         represented as an array of {@link SnapshottableDirectoryStatus}. If
312   *         {@code userName} is null, return all the snapshottable dirs.
313   */
314  public SnapshottableDirectoryStatus[] getSnapshottableDirListing(
315      String userName) {
316    if (snapshottables.isEmpty()) {
317      return null;
318    }
319    
320    List<SnapshottableDirectoryStatus> statusList = 
321        new ArrayList<SnapshottableDirectoryStatus>();
322    for (INodeDirectorySnapshottable dir : snapshottables.values()) {
323      if (userName == null || userName.equals(dir.getUserName())) {
324        SnapshottableDirectoryStatus status = new SnapshottableDirectoryStatus(
325            dir.getModificationTime(), dir.getAccessTime(),
326            dir.getFsPermission(), dir.getUserName(), dir.getGroupName(),
327            dir.getLocalNameBytes(), dir.getId(), dir.getChildrenNum(null),
328            dir.getNumSnapshots(),
329            dir.getSnapshotQuota(), dir.getParent() == null ? 
330                DFSUtil.EMPTY_BYTES : 
331                DFSUtil.string2Bytes(dir.getParent().getFullPathName()));
332        statusList.add(status);
333      }
334    }
335    Collections.sort(statusList, SnapshottableDirectoryStatus.COMPARATOR);
336    return statusList.toArray(
337        new SnapshottableDirectoryStatus[statusList.size()]);
338  }
339  
340  /**
341   * Compute the difference between two snapshots of a directory, or between a
342   * snapshot of the directory and its current tree.
343   */
344  public SnapshotDiffInfo diff(final String path, final String from,
345      final String to) throws IOException {
346    if ((from == null || from.isEmpty())
347        && (to == null || to.isEmpty())) {
348      // both fromSnapshot and toSnapshot indicate the current tree
349      return null;
350    }
351
352    // Find the source root directory path where the snapshots were taken.
353    // All the check for path has been included in the valueOf method.
354    INodesInPath inodesInPath = fsdir.getINodesInPath4Write(path.toString());
355    final INodeDirectorySnapshottable snapshotRoot = INodeDirectorySnapshottable
356        .valueOf(inodesInPath.getLastINode(), path);
357    
358    return snapshotRoot.computeDiff(from, to);
359  }
360  
361  public void clearSnapshottableDirs() {
362    snapshottables.clear();
363  }
364
365  /**
366   * Returns the maximum allowable snapshot ID based on the bit width of the
367   * snapshot ID.
368   *
369   * @return maximum allowable snapshot ID.
370   */
371   public int getMaxSnapshotID() {
372    return ((1 << SNAPSHOT_ID_BIT_WIDTH) - 1);
373  }
374}