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