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