|
@@ -21,7 +21,6 @@ package org.apache.hadoop.hdfs.server.namenode;
|
|
|
import java.io.BufferedInputStream;
|
|
|
import java.io.BufferedOutputStream;
|
|
|
import java.io.ByteArrayInputStream;
|
|
|
-import java.io.ByteArrayOutputStream;
|
|
|
import java.io.File;
|
|
|
import java.io.FileInputStream;
|
|
|
import java.io.FileOutputStream;
|
|
@@ -31,7 +30,6 @@ import java.io.OutputStream;
|
|
|
import java.io.RandomAccessFile;
|
|
|
import java.nio.ByteBuffer;
|
|
|
import java.nio.channels.FileChannel;
|
|
|
-import java.security.DigestInputStream;
|
|
|
import java.security.DigestOutputStream;
|
|
|
import java.security.MessageDigest;
|
|
|
import java.util.Arrays;
|
|
@@ -43,8 +41,10 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection;
|
|
|
+import org.apache.hadoop.hdfs.util.MD5FileUtils;
|
|
|
import org.apache.hadoop.io.MD5Hash;
|
|
|
import org.apache.hadoop.io.compress.CompressionCodec;
|
|
|
+import org.apache.hadoop.io.compress.CompressorStream;
|
|
|
|
|
|
import com.google.common.io.LimitInputStream;
|
|
|
import com.google.protobuf.CodedOutputStream;
|
|
@@ -84,6 +84,7 @@ final class FSImageFormatProtobuf {
|
|
|
}
|
|
|
|
|
|
void load(File file) throws IOException {
|
|
|
+ imgDigest = MD5FileUtils.computeMd5ForFile(file);
|
|
|
RandomAccessFile raFile = new RandomAccessFile(file, "r");
|
|
|
FileInputStream fin = new FileInputStream(file);
|
|
|
try {
|
|
@@ -139,7 +140,6 @@ final class FSImageFormatProtobuf {
|
|
|
}
|
|
|
FileSummary summary = loadSummary(raFile);
|
|
|
|
|
|
- MessageDigest digester = MD5Hash.getDigester();
|
|
|
FileChannel channel = fin.getChannel();
|
|
|
|
|
|
FSImageFormatPBINode.Loader inodeLoader = new FSImageFormatPBINode.Loader(
|
|
@@ -147,8 +147,8 @@ final class FSImageFormatProtobuf {
|
|
|
|
|
|
for (FileSummary.Section s : summary.getSectionsList()) {
|
|
|
channel.position(s.getOffset());
|
|
|
- InputStream in = new DigestInputStream(new BufferedInputStream(
|
|
|
- new LimitInputStream(fin, s.getLength())), digester);
|
|
|
+ InputStream in = new BufferedInputStream(new LimitInputStream(fin,
|
|
|
+ s.getLength()));
|
|
|
|
|
|
if (summary.hasCodec()) {
|
|
|
// read compression related info
|
|
@@ -179,10 +179,6 @@ final class FSImageFormatProtobuf {
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- updateDigestForFileSummary(summary, digester);
|
|
|
-
|
|
|
- imgDigest = new MD5Hash(digester.digest());
|
|
|
}
|
|
|
|
|
|
private void loadNameSystemSection(InputStream in, FileSummary.Section sections)
|
|
@@ -204,6 +200,8 @@ final class FSImageFormatProtobuf {
|
|
|
private FileChannel fileChannel;
|
|
|
// OutputStream for the section data
|
|
|
private OutputStream sectionOutputStream;
|
|
|
+ private CompressionCodec codec;
|
|
|
+ private OutputStream underlyingOutputStream;
|
|
|
|
|
|
Saver(SaveNamespaceContext context) {
|
|
|
this.context = context;
|
|
@@ -216,15 +214,29 @@ final class FSImageFormatProtobuf {
|
|
|
void commitSection(FileSummary.Builder summary, SectionName name)
|
|
|
throws IOException {
|
|
|
long oldOffset = currentOffset;
|
|
|
- sectionOutputStream.flush();
|
|
|
+ flushSectionOutputStream();
|
|
|
+
|
|
|
+ if (codec != null) {
|
|
|
+ sectionOutputStream = codec.createOutputStream(underlyingOutputStream);
|
|
|
+ } else {
|
|
|
+ sectionOutputStream = underlyingOutputStream;
|
|
|
+ }
|
|
|
long length = fileChannel.position() - oldOffset;
|
|
|
summary.addSections(FileSummary.Section.newBuilder().setName(name.name)
|
|
|
.setLength(length).setOffset(currentOffset));
|
|
|
currentOffset += length;
|
|
|
}
|
|
|
|
|
|
+ private void flushSectionOutputStream() throws IOException {
|
|
|
+ if (codec != null) {
|
|
|
+ ((CompressorStream) sectionOutputStream).finish();
|
|
|
+ }
|
|
|
+ sectionOutputStream.flush();
|
|
|
+ }
|
|
|
+
|
|
|
void save(File file, FSImageCompression compression) throws IOException {
|
|
|
FileOutputStream fout = new FileOutputStream(file);
|
|
|
+ fileChannel = fout.getChannel();
|
|
|
try {
|
|
|
saveInternal(fout, compression);
|
|
|
} finally {
|
|
@@ -232,60 +244,60 @@ final class FSImageFormatProtobuf {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void saveFileSummary(FileOutputStream fout, FileSummary summary)
|
|
|
+ private static void saveFileSummary(OutputStream out, FileSummary summary)
|
|
|
throws IOException {
|
|
|
- summary.writeDelimitedTo(fout);
|
|
|
+ summary.writeDelimitedTo(out);
|
|
|
int length = getOndiskTrunkSize(summary);
|
|
|
byte[] lengthBytes = new byte[4];
|
|
|
ByteBuffer.wrap(lengthBytes).asIntBuffer().put(length);
|
|
|
- fout.write(lengthBytes);
|
|
|
+ out.write(lengthBytes);
|
|
|
}
|
|
|
|
|
|
- private void saveInodes(OutputStream out, FileSummary.Builder summary)
|
|
|
- throws IOException {
|
|
|
+ private void saveInodes(FileSummary.Builder summary) throws IOException {
|
|
|
FSImageFormatPBINode.Saver saver = new FSImageFormatPBINode.Saver(this,
|
|
|
- out, summary);
|
|
|
- saver.serializeINodeSection();
|
|
|
- saver.serializeINodeDirectorySection();
|
|
|
- saver.serializeFilesUCSection();
|
|
|
+ summary);
|
|
|
+ saver.serializeINodeSection(sectionOutputStream);
|
|
|
+ saver.serializeINodeDirectorySection(sectionOutputStream);
|
|
|
+ saver.serializeFilesUCSection(sectionOutputStream);
|
|
|
}
|
|
|
|
|
|
private void saveInternal(FileOutputStream fout,
|
|
|
FSImageCompression compression) throws IOException {
|
|
|
- fout.write(MAGIC_HEADER);
|
|
|
- fileChannel = fout.getChannel();
|
|
|
-
|
|
|
MessageDigest digester = MD5Hash.getDigester();
|
|
|
- OutputStream out = new DigestOutputStream(new BufferedOutputStream(fout),
|
|
|
- digester);
|
|
|
+ underlyingOutputStream = new DigestOutputStream(new BufferedOutputStream(
|
|
|
+ fout), digester);
|
|
|
+ underlyingOutputStream.write(MAGIC_HEADER);
|
|
|
+
|
|
|
+ fileChannel = fout.getChannel();
|
|
|
|
|
|
FileSummary.Builder b = FileSummary.newBuilder()
|
|
|
.setOndiskVersion(FILE_VERSION)
|
|
|
.setLayoutVersion(LayoutVersion.getCurrentLayoutVersion());
|
|
|
|
|
|
- CompressionCodec codec = compression.getImageCodec();
|
|
|
+ codec = compression.getImageCodec();
|
|
|
if (codec != null) {
|
|
|
b.setCodec(codec.getClass().getCanonicalName());
|
|
|
- sectionOutputStream = codec.createOutputStream(out);
|
|
|
+ sectionOutputStream = codec.createOutputStream(underlyingOutputStream);
|
|
|
} else {
|
|
|
- sectionOutputStream = out;
|
|
|
+ sectionOutputStream = underlyingOutputStream;
|
|
|
}
|
|
|
|
|
|
- saveNameSystemSection(sectionOutputStream, b);
|
|
|
- saveInodes(sectionOutputStream, b);
|
|
|
+ saveNameSystemSection(b);
|
|
|
+ saveInodes(b);
|
|
|
|
|
|
// Flush the buffered data into the file before appending the header
|
|
|
- out.flush();
|
|
|
+ flushSectionOutputStream();
|
|
|
|
|
|
FileSummary summary = b.build();
|
|
|
- saveFileSummary(fout, summary);
|
|
|
- updateDigestForFileSummary(summary, digester);
|
|
|
+ saveFileSummary(underlyingOutputStream, summary);
|
|
|
+ underlyingOutputStream.close();
|
|
|
savedDigest = new MD5Hash(digester.digest());
|
|
|
}
|
|
|
|
|
|
- private void saveNameSystemSection(OutputStream out,
|
|
|
+ private void saveNameSystemSection(
|
|
|
FileSummary.Builder summary) throws IOException {
|
|
|
final FSNamesystem fsn = context.getSourceNamesystem();
|
|
|
+ OutputStream out = sectionOutputStream;
|
|
|
NameSystemSection.Builder b = NameSystemSection.newBuilder()
|
|
|
.setGenstampV1(fsn.getGenerationStampV1())
|
|
|
.setGenstampV1Limit(fsn.getGenerationStampV1Limit())
|
|
@@ -335,19 +347,6 @@ final class FSImageFormatProtobuf {
|
|
|
+ s.getSerializedSize();
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Include the FileSummary when calculating the digest. This is required as the
|
|
|
- * code does not access the FSImage strictly in sequential order.
|
|
|
- */
|
|
|
- private static void updateDigestForFileSummary(FileSummary summary,
|
|
|
- MessageDigest digester) throws IOException {
|
|
|
- ByteArrayOutputStream o = new ByteArrayOutputStream();
|
|
|
- o.write(MAGIC_HEADER);
|
|
|
- summary.writeDelimitedTo(o);
|
|
|
- digester.update(o.toByteArray());
|
|
|
- }
|
|
|
-
|
|
|
private FSImageFormatProtobuf() {
|
|
|
}
|
|
|
-
|
|
|
}
|