|
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
import java.io.BufferedInputStream;
|
|
import java.io.BufferedInputStream;
|
|
import java.io.BufferedOutputStream;
|
|
import java.io.BufferedOutputStream;
|
|
|
|
+import java.io.ByteArrayInputStream;
|
|
import java.io.ByteArrayOutputStream;
|
|
import java.io.ByteArrayOutputStream;
|
|
import java.io.File;
|
|
import java.io.File;
|
|
import java.io.FileInputStream;
|
|
import java.io.FileInputStream;
|
|
@@ -27,6 +28,8 @@ import java.io.FileOutputStream;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.io.InputStream;
|
|
import java.io.InputStream;
|
|
import java.io.OutputStream;
|
|
import java.io.OutputStream;
|
|
|
|
+import java.io.RandomAccessFile;
|
|
|
|
+import java.nio.ByteBuffer;
|
|
import java.nio.channels.FileChannel;
|
|
import java.nio.channels.FileChannel;
|
|
import java.security.DigestInputStream;
|
|
import java.security.DigestInputStream;
|
|
import java.security.DigestOutputStream;
|
|
import java.security.DigestOutputStream;
|
|
@@ -38,60 +41,32 @@ import org.apache.commons.logging.LogFactory;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
|
|
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
|
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileHeader;
|
|
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection;
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection;
|
|
-import org.apache.hadoop.io.IOUtils;
|
|
|
|
import org.apache.hadoop.io.MD5Hash;
|
|
import org.apache.hadoop.io.MD5Hash;
|
|
import org.apache.hadoop.io.compress.CompressionCodec;
|
|
import org.apache.hadoop.io.compress.CompressionCodec;
|
|
|
|
|
|
-import com.google.common.base.Preconditions;
|
|
|
|
|
|
+import com.google.common.io.LimitInputStream;
|
|
import com.google.protobuf.CodedOutputStream;
|
|
import com.google.protobuf.CodedOutputStream;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Utility class to read / write fsimage in protobuf format.
|
|
* Utility class to read / write fsimage in protobuf format.
|
|
*/
|
|
*/
|
|
final class FSImageFormatProtobuf {
|
|
final class FSImageFormatProtobuf {
|
|
- private static final Log LOG = LogFactory
|
|
|
|
- .getLog(DelegationTokenSecretManager.class);
|
|
|
|
-
|
|
|
|
|
|
+ private static final Log LOG = LogFactory.getLog(FSImageFormatProtobuf.class);
|
|
|
|
+
|
|
static final byte[] MAGIC_HEADER = "HDFSIMG1".getBytes();
|
|
static final byte[] MAGIC_HEADER = "HDFSIMG1".getBytes();
|
|
private static final int FILE_VERSION = 1;
|
|
private static final int FILE_VERSION = 1;
|
|
- private static final int PRE_ALLOCATED_HEADER_SIZE = 1024;
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Supported section name
|
|
|
|
- */
|
|
|
|
- private enum SectionName {
|
|
|
|
- NS_INFO("NS_INFO");
|
|
|
|
-
|
|
|
|
- private static final SectionName[] values = SectionName.values();
|
|
|
|
- private final String name;
|
|
|
|
-
|
|
|
|
- private SectionName(String name) {
|
|
|
|
- this.name = name;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private static SectionName fromString(String name) {
|
|
|
|
- for (SectionName n : values) {
|
|
|
|
- if (n.name.equals(name))
|
|
|
|
- return n;
|
|
|
|
- }
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // Buffer size of when reading / writing fsimage
|
|
|
|
- public static final int DEFAULT_BUFFER_SIZE = 16 * 1024 * 1024;
|
|
|
|
-
|
|
|
|
|
|
+
|
|
static final class Loader implements FSImageFormat.AbstractLoader {
|
|
static final class Loader implements FSImageFormat.AbstractLoader {
|
|
|
|
+ private static final int MINIMUM_FILE_LENGTH = 8;
|
|
private final Configuration conf;
|
|
private final Configuration conf;
|
|
private final FSNamesystem fsn;
|
|
private final FSNamesystem fsn;
|
|
|
|
|
|
- /** The transaction ID of the last edit represented by the loaded file */
|
|
|
|
- private long imgTxId;
|
|
|
|
/** The MD5 sum of the loaded file */
|
|
/** The MD5 sum of the loaded file */
|
|
private MD5Hash imgDigest;
|
|
private MD5Hash imgDigest;
|
|
|
|
+ /** The transaction ID of the last edit represented by the loaded file */
|
|
|
|
+ private long imgTxId;
|
|
|
|
|
|
Loader(Configuration conf, FSNamesystem fsn) {
|
|
Loader(Configuration conf, FSNamesystem fsn) {
|
|
this.conf = conf;
|
|
this.conf = conf;
|
|
@@ -108,65 +83,106 @@ final class FSImageFormatProtobuf {
|
|
return imgTxId;
|
|
return imgTxId;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ void load(File file) throws IOException {
|
|
|
|
+ RandomAccessFile raFile = new RandomAccessFile(file, "r");
|
|
|
|
+ FileInputStream fin = new FileInputStream(file);
|
|
|
|
+ try {
|
|
|
|
+ loadInternal(raFile, fin);
|
|
|
|
+ } finally {
|
|
|
|
+ fin.close();
|
|
|
|
+ raFile.close();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private boolean checkFileFormat(RandomAccessFile file) throws IOException {
|
|
|
|
+ if (file.length() < MINIMUM_FILE_LENGTH)
|
|
|
|
+ return false;
|
|
|
|
+
|
|
|
|
+ byte[] magic = new byte[MAGIC_HEADER.length];
|
|
|
|
+ file.readFully(magic);
|
|
|
|
+ if (!Arrays.equals(MAGIC_HEADER, magic))
|
|
|
|
+ return false;
|
|
|
|
+
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private FileSummary loadSummary(RandomAccessFile file) throws IOException {
|
|
|
|
+ final int FILE_LENGTH_FIELD_SIZE = 4;
|
|
|
|
+ long fileLength = file.length();
|
|
|
|
+ file.seek(fileLength - FILE_LENGTH_FIELD_SIZE);
|
|
|
|
+ int summaryLength = file.readInt();
|
|
|
|
+ file.seek(fileLength - FILE_LENGTH_FIELD_SIZE - summaryLength);
|
|
|
|
+
|
|
|
|
+ byte[] summaryBytes = new byte[summaryLength];
|
|
|
|
+ file.readFully(summaryBytes);
|
|
|
|
+
|
|
|
|
+ FileSummary summary = FileSummary
|
|
|
|
+ .parseDelimitedFrom(new ByteArrayInputStream(summaryBytes));
|
|
|
|
+ if (summary.getOndiskVersion() != FILE_VERSION) {
|
|
|
|
+ throw new IOException("Unsupported file version "
|
|
|
|
+ + summary.getOndiskVersion());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (!LayoutVersion.supports(Feature.PROTOBUF_FORMAT,
|
|
|
|
+ summary.getLayoutVersion())) {
|
|
|
|
+ throw new IOException("Unsupported layout version "
|
|
|
|
+ + summary.getLayoutVersion());
|
|
|
|
+ }
|
|
|
|
+ return summary;
|
|
|
|
+ }
|
|
|
|
+
|
|
@SuppressWarnings("resource")
|
|
@SuppressWarnings("resource")
|
|
- public void load(FileInputStream fin) throws IOException {
|
|
|
|
- FileHeader header = loadHeader(new BufferedInputStream(fin));
|
|
|
|
|
|
+ private void loadInternal(RandomAccessFile raFile, FileInputStream fin)
|
|
|
|
+ throws IOException {
|
|
|
|
+ if (!checkFileFormat(raFile)) {
|
|
|
|
+ throw new IOException("Unrecognized file format");
|
|
|
|
+ }
|
|
|
|
+ FileSummary summary = loadSummary(raFile);
|
|
|
|
|
|
- fin.getChannel().position(header.getDataOffset());
|
|
|
|
MessageDigest digester = MD5Hash.getDigester();
|
|
MessageDigest digester = MD5Hash.getDigester();
|
|
- InputStream in = new DigestInputStream(new BufferedInputStream(fin,
|
|
|
|
- DEFAULT_BUFFER_SIZE), digester);
|
|
|
|
-
|
|
|
|
- if (header.hasCodec()) {
|
|
|
|
- // read compression related info
|
|
|
|
- FSImageCompression compression = FSImageCompression.createCompression(
|
|
|
|
- conf, header.getCodec());
|
|
|
|
- CompressionCodec imageCodec = compression.getImageCodec();
|
|
|
|
- if (header.getCodec() != null) {
|
|
|
|
- in = imageCodec.createInputStream(in);
|
|
|
|
|
|
+ FileChannel channel = fin.getChannel();
|
|
|
|
+
|
|
|
|
+ FSImageFormatPBINode.Loader inodeLoader = new FSImageFormatPBINode.Loader(
|
|
|
|
+ fsn);
|
|
|
|
+
|
|
|
|
+ for (FileSummary.Section s : summary.getSectionsList()) {
|
|
|
|
+ channel.position(s.getOffset());
|
|
|
|
+ InputStream in = new DigestInputStream(new BufferedInputStream(
|
|
|
|
+ new LimitInputStream(fin, s.getLength())), digester);
|
|
|
|
+
|
|
|
|
+ if (summary.hasCodec()) {
|
|
|
|
+ // read compression related info
|
|
|
|
+ FSImageCompression compression = FSImageCompression
|
|
|
|
+ .createCompression(conf, summary.getCodec());
|
|
|
|
+ CompressionCodec imageCodec = compression.getImageCodec();
|
|
|
|
+ if (summary.getCodec() != null) {
|
|
|
|
+ in = imageCodec.createInputStream(in);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
- }
|
|
|
|
|
|
|
|
- for (FileHeader.Section s : header.getSectionsList()) {
|
|
|
|
String n = s.getName();
|
|
String n = s.getName();
|
|
switch (SectionName.fromString(n)) {
|
|
switch (SectionName.fromString(n)) {
|
|
case NS_INFO:
|
|
case NS_INFO:
|
|
loadNameSystemSection(in, s);
|
|
loadNameSystemSection(in, s);
|
|
break;
|
|
break;
|
|
|
|
+ case INODE:
|
|
|
|
+ inodeLoader.loadINodeSection(in);
|
|
|
|
+ break;
|
|
|
|
+ case INODE_DIR:
|
|
|
|
+ inodeLoader.loadINodeDirectorySection(in);
|
|
|
|
+ break;
|
|
default:
|
|
default:
|
|
LOG.warn("Unregconized section " + n);
|
|
LOG.warn("Unregconized section " + n);
|
|
break;
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- updateDigestForFileHeader(header, digester);
|
|
|
|
|
|
+ updateDigestForFileSummary(summary, digester);
|
|
|
|
|
|
imgDigest = new MD5Hash(digester.digest());
|
|
imgDigest = new MD5Hash(digester.digest());
|
|
- in.close();
|
|
|
|
}
|
|
}
|
|
|
|
|
|
- private FileHeader loadHeader(InputStream fin) throws IOException {
|
|
|
|
- byte[] magic = new byte[MAGIC_HEADER.length];
|
|
|
|
- if (fin.read(magic) != magic.length
|
|
|
|
- || !Arrays.equals(magic, FSImageFormatProtobuf.MAGIC_HEADER)) {
|
|
|
|
- throw new IOException("Unrecognized FSImage");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- FileHeader header = FileHeader.parseDelimitedFrom(fin);
|
|
|
|
- if (header.getOndiskVersion() != FILE_VERSION) {
|
|
|
|
- throw new IOException("Unsupported file version "
|
|
|
|
- + header.getOndiskVersion());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (!LayoutVersion.supports(Feature.PROTOBUF_FORMAT,
|
|
|
|
- header.getLayoutVersion())) {
|
|
|
|
- throw new IOException("Unsupported layout version "
|
|
|
|
- + header.getLayoutVersion());
|
|
|
|
- }
|
|
|
|
- return header;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void loadNameSystemSection(InputStream in, FileHeader.Section header)
|
|
|
|
|
|
+ private void loadNameSystemSection(InputStream in, FileSummary.Section sections)
|
|
throws IOException {
|
|
throws IOException {
|
|
NameSystemSection s = NameSystemSection.parseDelimitedFrom(in);
|
|
NameSystemSection s = NameSystemSection.parseDelimitedFrom(in);
|
|
fsn.setGenerationStampV1(s.getGenstampV1());
|
|
fsn.setGenerationStampV1(s.getGenstampV1());
|
|
@@ -174,62 +190,98 @@ final class FSImageFormatProtobuf {
|
|
fsn.setGenerationStampV1Limit(s.getGenstampV1Limit());
|
|
fsn.setGenerationStampV1Limit(s.getGenstampV1Limit());
|
|
fsn.setLastAllocatedBlockId(s.getLastAllocatedBlockId());
|
|
fsn.setLastAllocatedBlockId(s.getLastAllocatedBlockId());
|
|
imgTxId = s.getTransactionId();
|
|
imgTxId = s.getTransactionId();
|
|
- long offset = header.getLength() - getOndiskTrunkSize(s);
|
|
|
|
- Preconditions.checkArgument(offset == 0);
|
|
|
|
- in.skip(offset);
|
|
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
static final class Saver {
|
|
static final class Saver {
|
|
- private final SaveNamespaceContext context;
|
|
|
|
|
|
+ final SaveNamespaceContext context;
|
|
|
|
+ private long currentOffset = MAGIC_HEADER.length;
|
|
private MD5Hash savedDigest;
|
|
private MD5Hash savedDigest;
|
|
|
|
|
|
|
|
+ private FileChannel fileChannel;
|
|
|
|
+ // OutputStream for the section data
|
|
|
|
+ private OutputStream sectionOutputStream;
|
|
|
|
+
|
|
Saver(SaveNamespaceContext context) {
|
|
Saver(SaveNamespaceContext context) {
|
|
this.context = context;
|
|
this.context = context;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public MD5Hash getSavedDigest() {
|
|
|
|
+ return savedDigest;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ void commitSection(FileSummary.Builder summary, SectionName name)
|
|
|
|
+ throws IOException {
|
|
|
|
+ long oldOffset = currentOffset;
|
|
|
|
+ sectionOutputStream.flush();
|
|
|
|
+ long length = fileChannel.position() - oldOffset;
|
|
|
|
+ summary.addSections(FileSummary.Section.newBuilder().setName(name.name)
|
|
|
|
+ .setLength(length).setOffset(currentOffset));
|
|
|
|
+ currentOffset += length;
|
|
|
|
+ }
|
|
|
|
+
|
|
void save(File file, FSImageCompression compression) throws IOException {
|
|
void save(File file, FSImageCompression compression) throws IOException {
|
|
- FileHeader.Builder b = FileHeader.newBuilder()
|
|
|
|
- .setOndiskVersion(FILE_VERSION)
|
|
|
|
- .setLayoutVersion(LayoutVersion.getCurrentLayoutVersion())
|
|
|
|
- .setDataOffset(PRE_ALLOCATED_HEADER_SIZE);
|
|
|
|
- MessageDigest digester = MD5Hash.getDigester();
|
|
|
|
- OutputStream out = null;
|
|
|
|
|
|
+ FileOutputStream fout = new FileOutputStream(file);
|
|
try {
|
|
try {
|
|
- FileOutputStream fout = new FileOutputStream(file);
|
|
|
|
- FileChannel channel = fout.getChannel();
|
|
|
|
|
|
+ saveInternal(fout, compression);
|
|
|
|
+ } finally {
|
|
|
|
+ fout.close();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void saveFileSummary(FileOutputStream fout, FileSummary summary)
|
|
|
|
+ throws IOException {
|
|
|
|
+ summary.writeDelimitedTo(fout);
|
|
|
|
+ int length = getOndiskTrunkSize(summary);
|
|
|
|
+ byte[] lengthBytes = new byte[4];
|
|
|
|
+ ByteBuffer.wrap(lengthBytes).asIntBuffer().put(length);
|
|
|
|
+ fout.write(lengthBytes);
|
|
|
|
+ }
|
|
|
|
|
|
- channel.position(PRE_ALLOCATED_HEADER_SIZE);
|
|
|
|
- out = new DigestOutputStream(new BufferedOutputStream(fout,
|
|
|
|
- DEFAULT_BUFFER_SIZE), digester);
|
|
|
|
|
|
+ private void saveInodes(OutputStream out, FileSummary.Builder summary)
|
|
|
|
+ throws IOException {
|
|
|
|
+ FSImageFormatPBINode.Saver saver = new FSImageFormatPBINode.Saver(this,
|
|
|
|
+ out, summary);
|
|
|
|
+ saver.serializeINodeSection();
|
|
|
|
+ saver.serializeINodeDirectorySection();
|
|
|
|
+ }
|
|
|
|
|
|
- CompressionCodec codec = compression.getImageCodec();
|
|
|
|
- if (codec != null) {
|
|
|
|
- b.setCodec(codec.getClass().getCanonicalName());
|
|
|
|
- out = codec.createOutputStream(out);
|
|
|
|
- }
|
|
|
|
|
|
+ private void saveInternal(FileOutputStream fout,
|
|
|
|
+ FSImageCompression compression) throws IOException {
|
|
|
|
+ fout.write(MAGIC_HEADER);
|
|
|
|
+ fileChannel = fout.getChannel();
|
|
|
|
|
|
- save(out, b);
|
|
|
|
- out.flush();
|
|
|
|
- channel.position(0);
|
|
|
|
- FileHeader header = b.build();
|
|
|
|
- Preconditions.checkState(MAGIC_HEADER.length
|
|
|
|
- + getOndiskTrunkSize(header) < PRE_ALLOCATED_HEADER_SIZE,
|
|
|
|
- "Insufficient space to write file header");
|
|
|
|
- fout.write(MAGIC_HEADER);
|
|
|
|
- header.writeDelimitedTo(fout);
|
|
|
|
- updateDigestForFileHeader(header, digester);
|
|
|
|
- savedDigest = new MD5Hash(digester.digest());
|
|
|
|
- } finally {
|
|
|
|
- IOUtils.cleanup(LOG, out);
|
|
|
|
|
|
+ MessageDigest digester = MD5Hash.getDigester();
|
|
|
|
+ OutputStream out = new DigestOutputStream(new BufferedOutputStream(fout),
|
|
|
|
+ digester);
|
|
|
|
+
|
|
|
|
+ FileSummary.Builder b = FileSummary.newBuilder()
|
|
|
|
+ .setOndiskVersion(FILE_VERSION)
|
|
|
|
+ .setLayoutVersion(LayoutVersion.getCurrentLayoutVersion());
|
|
|
|
+
|
|
|
|
+ CompressionCodec codec = compression.getImageCodec();
|
|
|
|
+ if (codec != null) {
|
|
|
|
+ b.setCodec(codec.getClass().getCanonicalName());
|
|
|
|
+ sectionOutputStream = codec.createOutputStream(out);
|
|
|
|
+ } else {
|
|
|
|
+ sectionOutputStream = out;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ saveNameSystemSection(sectionOutputStream, b);
|
|
|
|
+ saveInodes(sectionOutputStream, b);
|
|
|
|
+
|
|
|
|
+ // Flush the buffered data into the file before appending the header
|
|
|
|
+ out.flush();
|
|
|
|
+
|
|
|
|
+ FileSummary summary = b.build();
|
|
|
|
+ saveFileSummary(fout, summary);
|
|
|
|
+ updateDigestForFileSummary(summary, digester);
|
|
|
|
+ savedDigest = new MD5Hash(digester.digest());
|
|
}
|
|
}
|
|
|
|
|
|
- private void save(OutputStream out, FileHeader.Builder headers)
|
|
|
|
- throws IOException {
|
|
|
|
|
|
+ private void saveNameSystemSection(OutputStream out,
|
|
|
|
+ FileSummary.Builder summary) throws IOException {
|
|
final FSNamesystem fsn = context.getSourceNamesystem();
|
|
final FSNamesystem fsn = context.getSourceNamesystem();
|
|
- FileHeader.Section.Builder sectionHeader = FileHeader.Section
|
|
|
|
- .newBuilder().setName(SectionName.NS_INFO.name);
|
|
|
|
NameSystemSection.Builder b = NameSystemSection.newBuilder()
|
|
NameSystemSection.Builder b = NameSystemSection.newBuilder()
|
|
.setGenstampV1(fsn.getGenerationStampV1())
|
|
.setGenstampV1(fsn.getGenerationStampV1())
|
|
.setGenstampV1Limit(fsn.getGenerationStampV1Limit())
|
|
.setGenstampV1Limit(fsn.getGenerationStampV1Limit())
|
|
@@ -245,33 +297,52 @@ final class FSImageFormatProtobuf {
|
|
b.setNamespaceId(fsn.unprotectedGetNamespaceInfo().getNamespaceID());
|
|
b.setNamespaceId(fsn.unprotectedGetNamespaceInfo().getNamespaceID());
|
|
NameSystemSection s = b.build();
|
|
NameSystemSection s = b.build();
|
|
s.writeDelimitedTo(out);
|
|
s.writeDelimitedTo(out);
|
|
- sectionHeader.setLength(getOndiskTrunkSize(s));
|
|
|
|
- headers.addSections(sectionHeader);
|
|
|
|
- }
|
|
|
|
|
|
|
|
- public MD5Hash getSavedDigest() {
|
|
|
|
- return savedDigest;
|
|
|
|
|
|
+ commitSection(summary, SectionName.NS_INFO);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Supported section name
|
|
|
|
+ */
|
|
|
|
+ enum SectionName {
|
|
|
|
+ INODE("INODE"), INODE_DIR("INODE_DIR"), NS_INFO("NS_INFO");
|
|
|
|
+
|
|
|
|
+ private static final SectionName[] values = SectionName.values();
|
|
|
|
+
|
|
|
|
+ private static SectionName fromString(String name) {
|
|
|
|
+ for (SectionName n : values) {
|
|
|
|
+ if (n.name.equals(name))
|
|
|
|
+ return n;
|
|
|
|
+ }
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private final String name;
|
|
|
|
+
|
|
|
|
+ private SectionName(String name) {
|
|
|
|
+ this.name = name;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
private static int getOndiskTrunkSize(com.google.protobuf.GeneratedMessage s) {
|
|
private static int getOndiskTrunkSize(com.google.protobuf.GeneratedMessage s) {
|
|
return CodedOutputStream.computeRawVarint32Size(s.getSerializedSize())
|
|
return CodedOutputStream.computeRawVarint32Size(s.getSerializedSize())
|
|
+ s.getSerializedSize();
|
|
+ s.getSerializedSize();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Include the FileHeader when calculating the digest. This is required as the
|
|
|
|
|
|
+ * Include the FileSummary when calculating the digest. This is required as the
|
|
* code does not access the FSImage strictly in sequential order.
|
|
* code does not access the FSImage strictly in sequential order.
|
|
*/
|
|
*/
|
|
- private static void updateDigestForFileHeader(FileHeader header,
|
|
|
|
|
|
+ private static void updateDigestForFileSummary(FileSummary summary,
|
|
MessageDigest digester) throws IOException {
|
|
MessageDigest digester) throws IOException {
|
|
ByteArrayOutputStream o = new ByteArrayOutputStream();
|
|
ByteArrayOutputStream o = new ByteArrayOutputStream();
|
|
o.write(MAGIC_HEADER);
|
|
o.write(MAGIC_HEADER);
|
|
- header.writeDelimitedTo(o);
|
|
|
|
|
|
+ summary.writeDelimitedTo(o);
|
|
digester.update(o.toByteArray());
|
|
digester.update(o.toByteArray());
|
|
}
|
|
}
|
|
|
|
|
|
private FSImageFormatProtobuf() {
|
|
private FSImageFormatProtobuf() {
|
|
}
|
|
}
|
|
|
|
|
|
-}
|
|
|
|
|
|
+}
|