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