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.HashMap;
025import java.util.List;
026import java.util.Map;
027
028import org.apache.hadoop.hdfs.DFSUtil;
029import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
030import org.apache.hadoop.hdfs.server.namenode.FSImageFormat.Loader;
031import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
032import org.apache.hadoop.hdfs.server.namenode.INode;
033import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
034import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
035import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
036import org.apache.hadoop.hdfs.server.namenode.INodeFile;
037import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
038import org.apache.hadoop.hdfs.server.namenode.INodeReference;
039import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
040import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiffList;
041import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
042import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiffList;
043import org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff;
044import org.apache.hadoop.hdfs.util.Diff.ListType;
045import org.apache.hadoop.hdfs.util.ReadOnlyList;
046
047/**
048 * A helper class defining static methods for reading/writing snapshot related
049 * information from/to FSImage.
050 */
051public class SnapshotFSImageFormat {
052  /**
053   * Save snapshots and snapshot quota for a snapshottable directory.
054   * @param current The directory that the snapshots belongs to.
055   * @param out The {@link DataOutput} to write.
056   * @throws IOException
057   */
058  public static void saveSnapshots(INodeDirectorySnapshottable current,
059      DataOutput out) throws IOException {
060    // list of snapshots in snapshotsByNames
061    ReadOnlyList<Snapshot> snapshots = current.getSnapshotsByNames();
062    out.writeInt(snapshots.size());
063    for (Snapshot s : snapshots) {
064      // write the snapshot id
065      out.writeInt(s.getId());
066    }
067    // snapshot quota
068    out.writeInt(current.getSnapshotQuota());
069  }
070  
071  /**
072   * Save SnapshotDiff list for an INodeDirectoryWithSnapshot.
073   * @param sNode The directory that the SnapshotDiff list belongs to.
074   * @param out The {@link DataOutput} to write.
075   */
076  private static <N extends INode, A extends INodeAttributes, D extends AbstractINodeDiff<N, A, D>>
077      void saveINodeDiffs(final AbstractINodeDiffList<N, A, D> diffs,
078      final DataOutput out, ReferenceMap referenceMap) throws IOException {
079    // Record the diffs in reversed order, so that we can find the correct
080    // reference for INodes in the created list when loading the FSImage
081    if (diffs == null) {
082      out.writeInt(-1); // no diffs
083    } else {
084      final List<D> list = diffs.asList();
085      final int size = list.size();
086      out.writeInt(size);
087      for (int i = size - 1; i >= 0; i--) {
088        list.get(i).write(out, referenceMap);
089      }
090    }
091  }
092  
093  public static void saveDirectoryDiffList(final INodeDirectory dir,
094      final DataOutput out, final ReferenceMap referenceMap
095      ) throws IOException {
096    saveINodeDiffs(dir instanceof INodeDirectoryWithSnapshot?
097        ((INodeDirectoryWithSnapshot)dir).getDiffs(): null, out, referenceMap);
098  }
099  
100  public static void saveFileDiffList(final INodeFile file,
101      final DataOutput out) throws IOException {
102    saveINodeDiffs(file instanceof FileWithSnapshot?
103        ((FileWithSnapshot)file).getDiffs(): null, out, null);
104  }
105
106  public static FileDiffList loadFileDiffList(DataInput in,
107      FSImageFormat.Loader loader) throws IOException {
108    final int size = in.readInt();
109    if (size == -1) {
110      return null;
111    } else {
112      final FileDiffList diffs = new FileDiffList();
113      FileDiff posterior = null;
114      for(int i = 0; i < size; i++) {
115        final FileDiff d = loadFileDiff(posterior, in, loader);
116        diffs.addFirst(d);
117        posterior = d;
118      }
119      return diffs;
120    }
121  }
122
123  private static FileDiff loadFileDiff(FileDiff posterior, DataInput in,
124      FSImageFormat.Loader loader) throws IOException {
125    // 1. Read the full path of the Snapshot root to identify the Snapshot
126    final Snapshot snapshot = loader.getSnapshot(in);
127
128    // 2. Load file size
129    final long fileSize = in.readLong();
130    
131    // 3. Load snapshotINode 
132    final INodeFileAttributes snapshotINode = in.readBoolean()?
133        loader.loadINodeFileAttributes(in): null;
134    
135    return new FileDiff(snapshot, snapshotINode, posterior, fileSize);
136  }
137
138  /**
139   * Load a node stored in the created list from fsimage.
140   * @param createdNodeName The name of the created node.
141   * @param parent The directory that the created list belongs to.
142   * @return The created node.
143   */
144  private static INode loadCreated(byte[] createdNodeName,
145      INodeDirectoryWithSnapshot parent) throws IOException {
146    // the INode in the created list should be a reference to another INode
147    // in posterior SnapshotDiffs or one of the current children
148    for (DirectoryDiff postDiff : parent.getDiffs()) {
149      final INode d = postDiff.getChildrenDiff().search(ListType.DELETED,
150          createdNodeName);
151      if (d != null) {
152        return d;
153      } // else go to the next SnapshotDiff
154    } 
155    // use the current child
156    INode currentChild = parent.getChild(createdNodeName, null);
157    if (currentChild == null) {
158      throw new IOException("Cannot find an INode associated with the INode "
159          + DFSUtil.bytes2String(createdNodeName)
160          + " in created list while loading FSImage.");
161    }
162    return currentChild;
163  }
164  
165  /**
166   * Load the created list from fsimage.
167   * @param parent The directory that the created list belongs to.
168   * @param in The {@link DataInput} to read.
169   * @return The created list.
170   */
171  private static List<INode> loadCreatedList(INodeDirectoryWithSnapshot parent,
172      DataInput in) throws IOException {
173    // read the size of the created list
174    int createdSize = in.readInt();
175    List<INode> createdList = new ArrayList<INode>(createdSize);
176    for (int i = 0; i < createdSize; i++) {
177      byte[] createdNodeName = FSImageSerialization.readLocalName(in);
178      INode created = loadCreated(createdNodeName, parent);
179      createdList.add(created);
180    }
181    return createdList;
182  }
183    
184  /**
185   * Load the deleted list from the fsimage.
186   * 
187   * @param parent The directory that the deleted list belongs to.
188   * @param createdList The created list associated with the deleted list in 
189   *                    the same Diff.
190   * @param in The {@link DataInput} to read.
191   * @param loader The {@link Loader} instance.
192   * @return The deleted list.
193   */
194  private static List<INode> loadDeletedList(INodeDirectoryWithSnapshot parent,
195      List<INode> createdList, DataInput in, FSImageFormat.Loader loader)
196      throws IOException {
197    int deletedSize = in.readInt();
198    List<INode> deletedList = new ArrayList<INode>(deletedSize);
199    for (int i = 0; i < deletedSize; i++) {
200      final INode deleted = loader.loadINodeWithLocalName(true, in, true);
201      deletedList.add(deleted);
202      // set parent: the parent field of an INode in the deleted list is not 
203      // useful, but set the parent here to be consistent with the original 
204      // fsdir tree.
205      deleted.setParent(parent);
206      if (deleted.isFile()) {
207        loader.updateBlocksMap(deleted.asFile());
208      }
209    }
210    return deletedList;
211  }
212  
213  /**
214   * Load snapshots and snapshotQuota for a Snapshottable directory.
215   * @param snapshottableParent The snapshottable directory for loading.
216   * @param numSnapshots The number of snapshots that the directory has.
217   * @param in The {@link DataInput} instance to read.
218   * @param loader The {@link Loader} instance that this loading procedure is 
219   *               using.
220   */
221  public static void loadSnapshotList(
222      INodeDirectorySnapshottable snapshottableParent, int numSnapshots,
223      DataInput in, FSImageFormat.Loader loader) throws IOException {
224    for (int i = 0; i < numSnapshots; i++) {
225      // read snapshots
226      final Snapshot s = loader.getSnapshot(in);
227      s.getRoot().setParent(snapshottableParent);
228      snapshottableParent.addSnapshot(s);
229    }
230    int snapshotQuota = in.readInt();
231    snapshottableParent.setSnapshotQuota(snapshotQuota);
232  }
233  
234  /**
235   * Load the {@link SnapshotDiff} list for the INodeDirectoryWithSnapshot
236   * directory.
237   * @param dir The snapshottable directory for loading.
238   * @param in The {@link DataInput} instance to read.
239   * @param loader The {@link Loader} instance that this loading procedure is 
240   *               using.
241   */
242  public static void loadDirectoryDiffList(INodeDirectory dir,
243      DataInput in, FSImageFormat.Loader loader) throws IOException {
244    final int size = in.readInt();
245    if (dir instanceof INodeDirectoryWithSnapshot) {
246      INodeDirectoryWithSnapshot withSnapshot = (INodeDirectoryWithSnapshot)dir;
247      DirectoryDiffList diffs = withSnapshot.getDiffs();
248      for (int i = 0; i < size; i++) {
249        diffs.addFirst(loadDirectoryDiff(withSnapshot, in, loader));
250      }
251    }
252  }
253  
254  /**
255   * Load the snapshotINode field of {@link AbstractINodeDiff}.
256   * @param snapshot The Snapshot associated with the {@link AbstractINodeDiff}.
257   * @param in The {@link DataInput} to read.
258   * @param loader The {@link Loader} instance that this loading procedure is 
259   *               using.
260   * @return The snapshotINode.
261   */
262  private static INodeDirectoryAttributes loadSnapshotINodeInDirectoryDiff(
263      Snapshot snapshot, DataInput in, FSImageFormat.Loader loader)
264      throws IOException {
265    // read the boolean indicating whether snapshotINode == Snapshot.Root
266    boolean useRoot = in.readBoolean();      
267    if (useRoot) {
268      return snapshot.getRoot();
269    } else {
270      // another boolean is used to indicate whether snapshotINode is non-null
271      return in.readBoolean()? loader.loadINodeDirectoryAttributes(in): null;
272    }
273  }
274   
275  /**
276   * Load {@link DirectoryDiff} from fsimage.
277   * @param parent The directory that the SnapshotDiff belongs to.
278   * @param in The {@link DataInput} instance to read.
279   * @param loader The {@link Loader} instance that this loading procedure is 
280   *               using.
281   * @return A {@link DirectoryDiff}.
282   */
283  private static DirectoryDiff loadDirectoryDiff(
284      INodeDirectoryWithSnapshot parent, DataInput in,
285      FSImageFormat.Loader loader) throws IOException {
286    // 1. Read the full path of the Snapshot root to identify the Snapshot
287    final Snapshot snapshot = loader.getSnapshot(in);
288
289    // 2. Load DirectoryDiff#childrenSize
290    int childrenSize = in.readInt();
291    
292    // 3. Load DirectoryDiff#snapshotINode 
293    INodeDirectoryAttributes snapshotINode = loadSnapshotINodeInDirectoryDiff(
294        snapshot, in, loader);
295    
296    // 4. Load the created list in SnapshotDiff#Diff
297    List<INode> createdList = loadCreatedList(parent, in);
298    
299    // 5. Load the deleted list in SnapshotDiff#Diff
300    List<INode> deletedList = loadDeletedList(parent, createdList, in, loader);
301    
302    // 6. Compose the SnapshotDiff
303    List<DirectoryDiff> diffs = parent.getDiffs().asList();
304    DirectoryDiff sdiff = new DirectoryDiff(snapshot, snapshotINode,
305        diffs.isEmpty() ? null : diffs.get(0),
306        childrenSize, createdList, deletedList);
307    return sdiff;
308  }
309  
310
311  /** A reference map for fsimage serialization. */
312  public static class ReferenceMap {
313    /**
314     * Used to indicate whether the reference node itself has been saved
315     */
316    private final Map<Long, INodeReference.WithCount> referenceMap
317        = new HashMap<Long, INodeReference.WithCount>();
318    /**
319     * Used to record whether the subtree of the reference node has been saved 
320     */
321    private final Map<Long, Long> dirMap = new HashMap<Long, Long>();
322
323    public void writeINodeReferenceWithCount(
324        INodeReference.WithCount withCount, DataOutput out,
325        boolean writeUnderConstruction) throws IOException {
326      final INode referred = withCount.getReferredINode();
327      final long id = withCount.getId();
328      final boolean firstReferred = !referenceMap.containsKey(id);
329      out.writeBoolean(firstReferred);
330
331      if (firstReferred) {
332        FSImageSerialization.saveINode2Image(referred, out,
333            writeUnderConstruction, this);
334        referenceMap.put(id, withCount);
335      } else {
336        out.writeLong(id);
337      }
338    }
339    
340    public boolean toProcessSubtree(long id) {
341      if (dirMap.containsKey(id)) {
342        return false;
343      } else {
344        dirMap.put(id, id);
345        return true;
346      }
347    }
348    
349    public INodeReference.WithCount loadINodeReferenceWithCount(
350        boolean isSnapshotINode, DataInput in, FSImageFormat.Loader loader
351        ) throws IOException {
352      final boolean firstReferred = in.readBoolean();
353
354      final INodeReference.WithCount withCount;
355      if (firstReferred) {
356        final INode referred = loader.loadINodeWithLocalName(isSnapshotINode,
357            in, true);
358        withCount = new INodeReference.WithCount(null, referred);
359        referenceMap.put(withCount.getId(), withCount);
360      } else {
361        final long id = in.readLong();
362        withCount = referenceMap.get(id);
363      }
364      return withCount;
365    }
366  }
367}