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