001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hdfs.server.namenode;
020
021import java.io.BufferedInputStream;
022import java.io.BufferedOutputStream;
023import java.io.File;
024import java.io.FileInputStream;
025import java.io.FileOutputStream;
026import java.io.IOException;
027import java.io.InputStream;
028import java.io.OutputStream;
029import java.io.RandomAccessFile;
030import java.nio.ByteBuffer;
031import java.nio.channels.FileChannel;
032import java.security.DigestOutputStream;
033import java.security.MessageDigest;
034import java.util.ArrayList;
035import java.util.Collections;
036import java.util.Comparator;
037import java.util.Map;
038import java.util.Map.Entry;
039import java.util.Set;
040
041import org.apache.commons.logging.Log;
042import org.apache.commons.logging.LogFactory;
043import org.apache.hadoop.classification.InterfaceAudience;
044import org.apache.hadoop.conf.Configuration;
045import org.apache.hadoop.hdfs.protocol.HdfsConstants;
046import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
047import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
048import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
049import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
050import org.apache.hadoop.hdfs.server.namenode.FsImageProto.CacheManagerSection;
051import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
052import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection;
053import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection;
054import org.apache.hadoop.hdfs.server.namenode.FsImageProto.StringTableSection;
055import org.apache.hadoop.hdfs.server.namenode.snapshot.FSImageFormatPBSnapshot;
056import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
057import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
058import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
059import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
060import org.apache.hadoop.hdfs.util.MD5FileUtils;
061import org.apache.hadoop.io.MD5Hash;
062import org.apache.hadoop.io.compress.CompressionCodec;
063import org.apache.hadoop.io.compress.CompressorStream;
064import org.apache.hadoop.util.LimitInputStream;
065import org.apache.hadoop.util.Time;
066
067import com.google.common.collect.Lists;
068import com.google.common.collect.Maps;
069import com.google.protobuf.CodedOutputStream;
070
071/**
072 * Utility class to read / write fsimage in protobuf format.
073 */
074@InterfaceAudience.Private
075public final class FSImageFormatProtobuf {
076  private static final Log LOG = LogFactory.getLog(FSImageFormatProtobuf.class);
077
078  public static final class LoaderContext {
079    private String[] stringTable;
080    private final ArrayList<INodeReference> refList = Lists.newArrayList();
081
082    public String[] getStringTable() {
083      return stringTable;
084    }
085
086    public ArrayList<INodeReference> getRefList() {
087      return refList;
088    }
089  }
090
091  public static final class SaverContext {
092    public static class DeduplicationMap<E> {
093      private final Map<E, Integer> map = Maps.newHashMap();
094      private DeduplicationMap() {}
095
096      static <T> DeduplicationMap<T> newMap() {
097        return new DeduplicationMap<T>();
098      }
099
100      int getId(E value) {
101        if (value == null) {
102          return 0;
103        }
104        Integer v = map.get(value);
105        if (v == null) {
106          int nv = map.size() + 1;
107          map.put(value, nv);
108          return nv;
109        }
110        return v;
111      }
112
113      int size() {
114        return map.size();
115      }
116
117      Set<Entry<E, Integer>> entrySet() {
118        return map.entrySet();
119      }
120    }
121    private final ArrayList<INodeReference> refList = Lists.newArrayList();
122
123    private final DeduplicationMap<String> stringMap = DeduplicationMap
124        .newMap();
125
126    public DeduplicationMap<String> getStringMap() {
127      return stringMap;
128    }
129
130    public ArrayList<INodeReference> getRefList() {
131      return refList;
132    }
133  }
134
135  public static final class Loader implements FSImageFormat.AbstractLoader {
136    static final int MINIMUM_FILE_LENGTH = 8;
137    private final Configuration conf;
138    private final FSNamesystem fsn;
139    private final LoaderContext ctx;
140    /** The MD5 sum of the loaded file */
141    private MD5Hash imgDigest;
142    /** The transaction ID of the last edit represented by the loaded file */
143    private long imgTxId;
144    /**
145     * Whether the image's layout version must be the same with
146     * {@link HdfsConstants#NAMENODE_LAYOUT_VERSION}. This is only set to true
147     * when we're doing (rollingUpgrade rollback).
148     */
149    private final boolean requireSameLayoutVersion;
150
151    Loader(Configuration conf, FSNamesystem fsn,
152        boolean requireSameLayoutVersion) {
153      this.conf = conf;
154      this.fsn = fsn;
155      this.ctx = new LoaderContext();
156      this.requireSameLayoutVersion = requireSameLayoutVersion;
157    }
158
159    @Override
160    public MD5Hash getLoadedImageMd5() {
161      return imgDigest;
162    }
163
164    @Override
165    public long getLoadedImageTxId() {
166      return imgTxId;
167    }
168
169    public LoaderContext getLoaderContext() {
170      return ctx;
171    }
172
173    void load(File file) throws IOException {
174      long start = Time.monotonicNow();
175      imgDigest = MD5FileUtils.computeMd5ForFile(file);
176      RandomAccessFile raFile = new RandomAccessFile(file, "r");
177      FileInputStream fin = new FileInputStream(file);
178      try {
179        loadInternal(raFile, fin);
180        long end = Time.monotonicNow();
181        LOG.info("Loaded FSImage in " + (end - start) / 1000 + " seconds.");
182      } finally {
183        fin.close();
184        raFile.close();
185      }
186    }
187
188    private void loadInternal(RandomAccessFile raFile, FileInputStream fin)
189        throws IOException {
190      if (!FSImageUtil.checkFileFormat(raFile)) {
191        throw new IOException("Unrecognized file format");
192      }
193      FileSummary summary = FSImageUtil.loadSummary(raFile);
194      if (requireSameLayoutVersion && summary.getLayoutVersion() !=
195          HdfsConstants.NAMENODE_LAYOUT_VERSION) {
196        throw new IOException("Image version " + summary.getLayoutVersion() +
197            " is not equal to the software version " +
198            HdfsConstants.NAMENODE_LAYOUT_VERSION);
199      }
200
201      FileChannel channel = fin.getChannel();
202
203      FSImageFormatPBINode.Loader inodeLoader = new FSImageFormatPBINode.Loader(
204          fsn, this);
205      FSImageFormatPBSnapshot.Loader snapshotLoader = new FSImageFormatPBSnapshot.Loader(
206          fsn, this);
207
208      ArrayList<FileSummary.Section> sections = Lists.newArrayList(summary
209          .getSectionsList());
210      Collections.sort(sections, new Comparator<FileSummary.Section>() {
211        @Override
212        public int compare(FileSummary.Section s1, FileSummary.Section s2) {
213          SectionName n1 = SectionName.fromString(s1.getName());
214          SectionName n2 = SectionName.fromString(s2.getName());
215          if (n1 == null) {
216            return n2 == null ? 0 : -1;
217          } else if (n2 == null) {
218            return -1;
219          } else {
220            return n1.ordinal() - n2.ordinal();
221          }
222        }
223      });
224
225      StartupProgress prog = NameNode.getStartupProgress();
226      /**
227       * beginStep() and the endStep() calls do not match the boundary of the
228       * sections. This is because that the current implementation only allows
229       * a particular step to be started for once.
230       */
231      Step currentStep = null;
232
233      for (FileSummary.Section s : sections) {
234        channel.position(s.getOffset());
235        InputStream in = new BufferedInputStream(new LimitInputStream(fin,
236            s.getLength()));
237
238        in = FSImageUtil.wrapInputStreamForCompression(conf,
239            summary.getCodec(), in);
240
241        String n = s.getName();
242
243        switch (SectionName.fromString(n)) {
244        case NS_INFO:
245          loadNameSystemSection(in);
246          break;
247        case STRING_TABLE:
248          loadStringTableSection(in);
249          break;
250        case INODE: {
251          currentStep = new Step(StepType.INODES);
252          prog.beginStep(Phase.LOADING_FSIMAGE, currentStep);
253          inodeLoader.loadINodeSection(in);
254        }
255          break;
256        case INODE_REFERENCE:
257          snapshotLoader.loadINodeReferenceSection(in);
258          break;
259        case INODE_DIR:
260          inodeLoader.loadINodeDirectorySection(in);
261          break;
262        case FILES_UNDERCONSTRUCTION:
263          inodeLoader.loadFilesUnderConstructionSection(in);
264          break;
265        case SNAPSHOT:
266          snapshotLoader.loadSnapshotSection(in);
267          break;
268        case SNAPSHOT_DIFF:
269          snapshotLoader.loadSnapshotDiffSection(in);
270          break;
271        case SECRET_MANAGER: {
272          prog.endStep(Phase.LOADING_FSIMAGE, currentStep);
273          Step step = new Step(StepType.DELEGATION_TOKENS);
274          prog.beginStep(Phase.LOADING_FSIMAGE, step);
275          loadSecretManagerSection(in);
276          prog.endStep(Phase.LOADING_FSIMAGE, step);
277        }
278          break;
279        case CACHE_MANAGER: {
280          Step step = new Step(StepType.CACHE_POOLS);
281          prog.beginStep(Phase.LOADING_FSIMAGE, step);
282          loadCacheManagerSection(in);
283          prog.endStep(Phase.LOADING_FSIMAGE, step);
284        }
285          break;
286        default:
287          LOG.warn("Unrecognized section " + n);
288          break;
289        }
290      }
291    }
292
293    private void loadNameSystemSection(InputStream in) throws IOException {
294      NameSystemSection s = NameSystemSection.parseDelimitedFrom(in);
295      BlockIdManager blockIdManager = fsn.getBlockIdManager();
296      blockIdManager.setGenerationStampV1(s.getGenstampV1());
297      blockIdManager.setGenerationStampV2(s.getGenstampV2());
298      blockIdManager.setGenerationStampV1Limit(s.getGenstampV1Limit());
299      blockIdManager.setLastAllocatedBlockId(s.getLastAllocatedBlockId());
300      imgTxId = s.getTransactionId();
301      if (s.hasRollingUpgradeStartTime()
302          && fsn.getFSImage().hasRollbackFSImage()) {
303        // we set the rollingUpgradeInfo only when we make sure we have the
304        // rollback image
305        fsn.setRollingUpgradeInfo(true, s.getRollingUpgradeStartTime());
306      }
307    }
308
309    private void loadStringTableSection(InputStream in) throws IOException {
310      StringTableSection s = StringTableSection.parseDelimitedFrom(in);
311      ctx.stringTable = new String[s.getNumEntry() + 1];
312      for (int i = 0; i < s.getNumEntry(); ++i) {
313        StringTableSection.Entry e = StringTableSection.Entry
314            .parseDelimitedFrom(in);
315        ctx.stringTable[e.getId()] = e.getStr();
316      }
317    }
318
319    private void loadSecretManagerSection(InputStream in) throws IOException {
320      SecretManagerSection s = SecretManagerSection.parseDelimitedFrom(in);
321      int numKeys = s.getNumKeys(), numTokens = s.getNumTokens();
322      ArrayList<SecretManagerSection.DelegationKey> keys = Lists
323          .newArrayListWithCapacity(numKeys);
324      ArrayList<SecretManagerSection.PersistToken> tokens = Lists
325          .newArrayListWithCapacity(numTokens);
326
327      for (int i = 0; i < numKeys; ++i)
328        keys.add(SecretManagerSection.DelegationKey.parseDelimitedFrom(in));
329
330      for (int i = 0; i < numTokens; ++i)
331        tokens.add(SecretManagerSection.PersistToken.parseDelimitedFrom(in));
332
333      fsn.loadSecretManagerState(s, keys, tokens);
334    }
335
336    private void loadCacheManagerSection(InputStream in) throws IOException {
337      CacheManagerSection s = CacheManagerSection.parseDelimitedFrom(in);
338      ArrayList<CachePoolInfoProto> pools = Lists.newArrayListWithCapacity(s
339          .getNumPools());
340      ArrayList<CacheDirectiveInfoProto> directives = Lists
341          .newArrayListWithCapacity(s.getNumDirectives());
342      for (int i = 0; i < s.getNumPools(); ++i)
343        pools.add(CachePoolInfoProto.parseDelimitedFrom(in));
344      for (int i = 0; i < s.getNumDirectives(); ++i)
345        directives.add(CacheDirectiveInfoProto.parseDelimitedFrom(in));
346      fsn.getCacheManager().loadState(
347          new CacheManager.PersistState(s, pools, directives));
348    }
349
350  }
351
352  public static final class Saver {
353    public static final int CHECK_CANCEL_INTERVAL = 4096;
354
355    private final SaveNamespaceContext context;
356    private final SaverContext saverContext;
357    private long currentOffset = FSImageUtil.MAGIC_HEADER.length;
358    private MD5Hash savedDigest;
359
360    private FileChannel fileChannel;
361    // OutputStream for the section data
362    private OutputStream sectionOutputStream;
363    private CompressionCodec codec;
364    private OutputStream underlyingOutputStream;
365
366    Saver(SaveNamespaceContext context) {
367      this.context = context;
368      this.saverContext = new SaverContext();
369    }
370
371    public MD5Hash getSavedDigest() {
372      return savedDigest;
373    }
374
375    public SaveNamespaceContext getContext() {
376      return context;
377    }
378
379    public SaverContext getSaverContext() {
380      return saverContext;
381    }
382
383    public void commitSection(FileSummary.Builder summary, SectionName name)
384        throws IOException {
385      long oldOffset = currentOffset;
386      flushSectionOutputStream();
387
388      if (codec != null) {
389        sectionOutputStream = codec.createOutputStream(underlyingOutputStream);
390      } else {
391        sectionOutputStream = underlyingOutputStream;
392      }
393      long length = fileChannel.position() - oldOffset;
394      summary.addSections(FileSummary.Section.newBuilder().setName(name.name)
395          .setLength(length).setOffset(currentOffset));
396      currentOffset += length;
397    }
398
399    private void flushSectionOutputStream() throws IOException {
400      if (codec != null) {
401        ((CompressorStream) sectionOutputStream).finish();
402      }
403      sectionOutputStream.flush();
404    }
405
406    void save(File file, FSImageCompression compression) throws IOException {
407      FileOutputStream fout = new FileOutputStream(file);
408      fileChannel = fout.getChannel();
409      try {
410        saveInternal(fout, compression, file.getAbsolutePath());
411      } finally {
412        fout.close();
413      }
414    }
415
416    private static void saveFileSummary(OutputStream out, FileSummary summary)
417        throws IOException {
418      summary.writeDelimitedTo(out);
419      int length = getOndiskTrunkSize(summary);
420      byte[] lengthBytes = new byte[4];
421      ByteBuffer.wrap(lengthBytes).asIntBuffer().put(length);
422      out.write(lengthBytes);
423    }
424
425    private void saveInodes(FileSummary.Builder summary) throws IOException {
426      FSImageFormatPBINode.Saver saver = new FSImageFormatPBINode.Saver(this,
427          summary);
428
429      saver.serializeINodeSection(sectionOutputStream);
430      saver.serializeINodeDirectorySection(sectionOutputStream);
431      saver.serializeFilesUCSection(sectionOutputStream);
432    }
433
434    private void saveSnapshots(FileSummary.Builder summary) throws IOException {
435      FSImageFormatPBSnapshot.Saver snapshotSaver = new FSImageFormatPBSnapshot.Saver(
436          this, summary, context, context.getSourceNamesystem());
437
438      snapshotSaver.serializeSnapshotSection(sectionOutputStream);
439      snapshotSaver.serializeSnapshotDiffSection(sectionOutputStream);
440      snapshotSaver.serializeINodeReferenceSection(sectionOutputStream);
441    }
442
443    private void saveInternal(FileOutputStream fout,
444        FSImageCompression compression, String filePath) throws IOException {
445      StartupProgress prog = NameNode.getStartupProgress();
446      MessageDigest digester = MD5Hash.getDigester();
447
448      underlyingOutputStream = new DigestOutputStream(new BufferedOutputStream(
449          fout), digester);
450      underlyingOutputStream.write(FSImageUtil.MAGIC_HEADER);
451
452      fileChannel = fout.getChannel();
453
454      FileSummary.Builder b = FileSummary.newBuilder()
455          .setOndiskVersion(FSImageUtil.FILE_VERSION)
456          .setLayoutVersion(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
457
458      codec = compression.getImageCodec();
459      if (codec != null) {
460        b.setCodec(codec.getClass().getCanonicalName());
461        sectionOutputStream = codec.createOutputStream(underlyingOutputStream);
462      } else {
463        sectionOutputStream = underlyingOutputStream;
464      }
465
466      saveNameSystemSection(b);
467      // Check for cancellation right after serializing the name system section.
468      // Some unit tests, such as TestSaveNamespace#testCancelSaveNameSpace
469      // depends on this behavior.
470      context.checkCancelled();
471
472      Step step = new Step(StepType.INODES, filePath);
473      prog.beginStep(Phase.SAVING_CHECKPOINT, step);
474      saveInodes(b);
475      saveSnapshots(b);
476      prog.endStep(Phase.SAVING_CHECKPOINT, step);
477
478      step = new Step(StepType.DELEGATION_TOKENS, filePath);
479      prog.beginStep(Phase.SAVING_CHECKPOINT, step);
480      saveSecretManagerSection(b);
481      prog.endStep(Phase.SAVING_CHECKPOINT, step);
482
483      step = new Step(StepType.CACHE_POOLS, filePath);
484      prog.beginStep(Phase.SAVING_CHECKPOINT, step);
485      saveCacheManagerSection(b);
486      prog.endStep(Phase.SAVING_CHECKPOINT, step);
487
488      saveStringTableSection(b);
489
490      // We use the underlyingOutputStream to write the header. Therefore flush
491      // the buffered stream (which is potentially compressed) first.
492      flushSectionOutputStream();
493
494      FileSummary summary = b.build();
495      saveFileSummary(underlyingOutputStream, summary);
496      underlyingOutputStream.close();
497      savedDigest = new MD5Hash(digester.digest());
498    }
499
500    private void saveSecretManagerSection(FileSummary.Builder summary)
501        throws IOException {
502      final FSNamesystem fsn = context.getSourceNamesystem();
503      DelegationTokenSecretManager.SecretManagerState state = fsn
504          .saveSecretManagerState();
505      state.section.writeDelimitedTo(sectionOutputStream);
506      for (SecretManagerSection.DelegationKey k : state.keys)
507        k.writeDelimitedTo(sectionOutputStream);
508
509      for (SecretManagerSection.PersistToken t : state.tokens)
510        t.writeDelimitedTo(sectionOutputStream);
511
512      commitSection(summary, SectionName.SECRET_MANAGER);
513    }
514
515    private void saveCacheManagerSection(FileSummary.Builder summary)
516        throws IOException {
517      final FSNamesystem fsn = context.getSourceNamesystem();
518      CacheManager.PersistState state = fsn.getCacheManager().saveState();
519      state.section.writeDelimitedTo(sectionOutputStream);
520
521      for (CachePoolInfoProto p : state.pools)
522        p.writeDelimitedTo(sectionOutputStream);
523
524      for (CacheDirectiveInfoProto p : state.directives)
525        p.writeDelimitedTo(sectionOutputStream);
526
527      commitSection(summary, SectionName.CACHE_MANAGER);
528    }
529
530    private void saveNameSystemSection(FileSummary.Builder summary)
531        throws IOException {
532      final FSNamesystem fsn = context.getSourceNamesystem();
533      OutputStream out = sectionOutputStream;
534      BlockIdManager blockIdManager = fsn.getBlockIdManager();
535      NameSystemSection.Builder b = NameSystemSection.newBuilder()
536          .setGenstampV1(blockIdManager.getGenerationStampV1())
537          .setGenstampV1Limit(blockIdManager.getGenerationStampV1Limit())
538          .setGenstampV2(blockIdManager.getGenerationStampV2())
539          .setLastAllocatedBlockId(blockIdManager.getLastAllocatedBlockId())
540          .setTransactionId(context.getTxId());
541
542      // We use the non-locked version of getNamespaceInfo here since
543      // the coordinating thread of saveNamespace already has read-locked
544      // the namespace for us. If we attempt to take another readlock
545      // from the actual saver thread, there's a potential of a
546      // fairness-related deadlock. See the comments on HDFS-2223.
547      b.setNamespaceId(fsn.unprotectedGetNamespaceInfo().getNamespaceID());
548      if (fsn.isRollingUpgrade()) {
549        b.setRollingUpgradeStartTime(fsn.getRollingUpgradeInfo().getStartTime());
550      }
551      NameSystemSection s = b.build();
552      s.writeDelimitedTo(out);
553
554      commitSection(summary, SectionName.NS_INFO);
555    }
556
557    private void saveStringTableSection(FileSummary.Builder summary)
558        throws IOException {
559      OutputStream out = sectionOutputStream;
560      StringTableSection.Builder b = StringTableSection.newBuilder()
561          .setNumEntry(saverContext.stringMap.size());
562      b.build().writeDelimitedTo(out);
563      for (Entry<String, Integer> e : saverContext.stringMap.entrySet()) {
564        StringTableSection.Entry.Builder eb = StringTableSection.Entry
565            .newBuilder().setId(e.getValue()).setStr(e.getKey());
566        eb.build().writeDelimitedTo(out);
567      }
568      commitSection(summary, SectionName.STRING_TABLE);
569    }
570  }
571
572  /**
573   * Supported section name. The order of the enum determines the order of
574   * loading.
575   */
576  public enum SectionName {
577    NS_INFO("NS_INFO"),
578    STRING_TABLE("STRING_TABLE"),
579    EXTENDED_ACL("EXTENDED_ACL"),
580    INODE("INODE"),
581    INODE_REFERENCE("INODE_REFERENCE"),
582    SNAPSHOT("SNAPSHOT"),
583    INODE_DIR("INODE_DIR"),
584    FILES_UNDERCONSTRUCTION("FILES_UNDERCONSTRUCTION"),
585    SNAPSHOT_DIFF("SNAPSHOT_DIFF"),
586    SECRET_MANAGER("SECRET_MANAGER"),
587    CACHE_MANAGER("CACHE_MANAGER");
588
589    private static final SectionName[] values = SectionName.values();
590
591    public static SectionName fromString(String name) {
592      for (SectionName n : values) {
593        if (n.name.equals(name))
594          return n;
595      }
596      return null;
597    }
598
599    private final String name;
600
601    private SectionName(String name) {
602      this.name = name;
603    }
604  }
605
606  private static int getOndiskTrunkSize(com.google.protobuf.GeneratedMessage s) {
607    return CodedOutputStream.computeRawVarint32Size(s.getSerializedSize())
608        + s.getSerializedSize();
609  }
610
611  private FSImageFormatProtobuf() {
612  }
613}