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