|
@@ -40,7 +40,11 @@ import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Map.Entry;
|
|
|
import java.util.Set;
|
|
|
+import java.util.Iterator;
|
|
|
+import java.util.concurrent.ExecutorService;
|
|
|
+import java.util.concurrent.Executors;
|
|
|
|
|
|
+import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
|
|
|
import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
|
|
|
import org.apache.hadoop.io.compress.CompressionOutputStream;
|
|
@@ -150,6 +154,8 @@ public final class FSImageFormatProtobuf {
|
|
|
*/
|
|
|
private final boolean requireSameLayoutVersion;
|
|
|
|
|
|
+ private File filename;
|
|
|
+
|
|
|
Loader(Configuration conf, FSNamesystem fsn,
|
|
|
boolean requireSameLayoutVersion) {
|
|
|
this.conf = conf;
|
|
@@ -229,6 +235,7 @@ public final class FSImageFormatProtobuf {
|
|
|
}
|
|
|
|
|
|
void load(File file) throws IOException {
|
|
|
+ filename = file;
|
|
|
long start = Time.monotonicNow();
|
|
|
DigestThread dt = new DigestThread(file);
|
|
|
dt.start();
|
|
@@ -250,6 +257,96 @@ public final class FSImageFormatProtobuf {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Given a FSImage FileSummary.section, return a LimitInput stream set to
|
|
|
+ * the starting position of the section and limited to the section length.
|
|
|
+ * @param section The FileSummary.Section containing the offset and length
|
|
|
+ * @param compressionCodec The compression codec in use, if any
|
|
|
+ * @return An InputStream for the given section
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ public InputStream getInputStreamForSection(FileSummary.Section section,
|
|
|
+ String compressionCodec)
|
|
|
+ throws IOException {
|
|
|
+ FileInputStream fin = new FileInputStream(filename);
|
|
|
+ FileChannel channel = fin.getChannel();
|
|
|
+ channel.position(section.getOffset());
|
|
|
+ InputStream in = new BufferedInputStream(new LimitInputStream(fin,
|
|
|
+ section.getLength()));
|
|
|
+
|
|
|
+ in = FSImageUtil.wrapInputStreamForCompression(conf,
|
|
|
+ compressionCodec, in);
|
|
|
+ return in;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Takes an ArrayList of Section's and removes all Section's whose
|
|
|
+ * name ends in _SUB, indicating they are sub-sections. The original
|
|
|
+ * array list is modified and a new list of the removed Section's is
|
|
|
+ * returned.
|
|
|
+ * @param sections Array List containing all Sections and Sub Sections
|
|
|
+ * in the image.
|
|
|
+ * @return ArrayList of the sections removed, or an empty list if none are
|
|
|
+ * removed.
|
|
|
+ */
|
|
|
+ private ArrayList<FileSummary.Section> getAndRemoveSubSections(
|
|
|
+ ArrayList<FileSummary.Section> sections) {
|
|
|
+ ArrayList<FileSummary.Section> subSections = new ArrayList<>();
|
|
|
+ Iterator<FileSummary.Section> iter = sections.iterator();
|
|
|
+ while (iter.hasNext()) {
|
|
|
+ FileSummary.Section s = iter.next();
|
|
|
+ String name = s.getName();
|
|
|
+ if (name.matches(".*_SUB$")) {
|
|
|
+ subSections.add(s);
|
|
|
+ iter.remove();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return subSections;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Given an ArrayList of Section's, return all Section's with the given
|
|
|
+ * name, or an empty list if none are found.
|
|
|
+ * @param sections ArrayList of the Section's to search though
|
|
|
+ * @param name The name of the Sections to search for
|
|
|
+ * @return ArrayList of the sections matching the given name
|
|
|
+ */
|
|
|
+ private ArrayList<FileSummary.Section> getSubSectionsOfName(
|
|
|
+ ArrayList<FileSummary.Section> sections, SectionName name) {
|
|
|
+ ArrayList<FileSummary.Section> subSec = new ArrayList<>();
|
|
|
+ for (FileSummary.Section s : sections) {
|
|
|
+ String n = s.getName();
|
|
|
+ SectionName sectionName = SectionName.fromString(n);
|
|
|
+ if (sectionName == name) {
|
|
|
+ subSec.add(s);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return subSec;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Checks the number of threads configured for parallel loading and
|
|
|
+ * return an ExecutorService with configured number of threads. If the
|
|
|
+ * thread count is set to less than 1, it will be reset to the default
|
|
|
+ * value
|
|
|
+ * @return ExecutorServie with the correct number of threads
|
|
|
+ */
|
|
|
+ private ExecutorService getParallelExecutorService() {
|
|
|
+ int threads = conf.getInt(DFSConfigKeys.DFS_IMAGE_PARALLEL_THREADS_KEY,
|
|
|
+ DFSConfigKeys.DFS_IMAGE_PARALLEL_THREADS_DEFAULT);
|
|
|
+ if (threads < 1) {
|
|
|
+ LOG.warn("Parallel is enabled and {} is set to {}. Setting to the " +
|
|
|
+ "default value {}", DFSConfigKeys.DFS_IMAGE_PARALLEL_THREADS_KEY,
|
|
|
+ threads, DFSConfigKeys.DFS_IMAGE_PARALLEL_THREADS_DEFAULT);
|
|
|
+ threads = DFSConfigKeys.DFS_IMAGE_PARALLEL_THREADS_DEFAULT;
|
|
|
+ }
|
|
|
+ ExecutorService executorService = Executors.newFixedThreadPool(
|
|
|
+ threads);
|
|
|
+ LOG.info("The fsimage will be loaded in parallel using {} threads",
|
|
|
+ threads);
|
|
|
+ return executorService;
|
|
|
+ }
|
|
|
+
|
|
|
private void loadInternal(RandomAccessFile raFile, FileInputStream fin)
|
|
|
throws IOException {
|
|
|
if (!FSImageUtil.checkFileFormat(raFile)) {
|
|
@@ -294,6 +391,14 @@ public final class FSImageFormatProtobuf {
|
|
|
* a particular step to be started for once.
|
|
|
*/
|
|
|
Step currentStep = null;
|
|
|
+ boolean loadInParallel = enableParallelSaveAndLoad(conf);
|
|
|
+
|
|
|
+ ExecutorService executorService = null;
|
|
|
+ ArrayList<FileSummary.Section> subSections =
|
|
|
+ getAndRemoveSubSections(sections);
|
|
|
+ if (loadInParallel) {
|
|
|
+ executorService = getParallelExecutorService();
|
|
|
+ }
|
|
|
|
|
|
for (FileSummary.Section s : sections) {
|
|
|
channel.position(s.getOffset());
|
|
@@ -308,6 +413,8 @@ public final class FSImageFormatProtobuf {
|
|
|
if (sectionName == null) {
|
|
|
throw new IOException("Unrecognized section " + n);
|
|
|
}
|
|
|
+
|
|
|
+ ArrayList<FileSummary.Section> stageSubSections;
|
|
|
switch (sectionName) {
|
|
|
case NS_INFO:
|
|
|
loadNameSystemSection(in);
|
|
@@ -318,14 +425,28 @@ public final class FSImageFormatProtobuf {
|
|
|
case INODE: {
|
|
|
currentStep = new Step(StepType.INODES);
|
|
|
prog.beginStep(Phase.LOADING_FSIMAGE, currentStep);
|
|
|
- inodeLoader.loadINodeSection(in, prog, currentStep);
|
|
|
+ stageSubSections = getSubSectionsOfName(
|
|
|
+ subSections, SectionName.INODE_SUB);
|
|
|
+ if (loadInParallel && (stageSubSections.size() > 0)) {
|
|
|
+ inodeLoader.loadINodeSectionInParallel(executorService,
|
|
|
+ stageSubSections, summary.getCodec(), prog, currentStep);
|
|
|
+ } else {
|
|
|
+ inodeLoader.loadINodeSection(in, prog, currentStep);
|
|
|
+ }
|
|
|
}
|
|
|
break;
|
|
|
case INODE_REFERENCE:
|
|
|
snapshotLoader.loadINodeReferenceSection(in);
|
|
|
break;
|
|
|
case INODE_DIR:
|
|
|
- inodeLoader.loadINodeDirectorySection(in);
|
|
|
+ stageSubSections = getSubSectionsOfName(
|
|
|
+ subSections, SectionName.INODE_DIR_SUB);
|
|
|
+ if (loadInParallel && stageSubSections.size() > 0) {
|
|
|
+ inodeLoader.loadINodeDirectorySectionInParallel(executorService,
|
|
|
+ stageSubSections, summary.getCodec());
|
|
|
+ } else {
|
|
|
+ inodeLoader.loadINodeDirectorySection(in);
|
|
|
+ }
|
|
|
break;
|
|
|
case FILES_UNDERCONSTRUCTION:
|
|
|
inodeLoader.loadFilesUnderConstructionSection(in);
|
|
@@ -362,6 +483,9 @@ public final class FSImageFormatProtobuf {
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
|
+ if (executorService != null) {
|
|
|
+ executorService.shutdown();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private void loadNameSystemSection(InputStream in) throws IOException {
|
|
@@ -450,12 +574,34 @@ public final class FSImageFormatProtobuf {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private static boolean enableParallelSaveAndLoad(Configuration conf) {
|
|
|
+ boolean loadInParallel =
|
|
|
+ conf.getBoolean(DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY,
|
|
|
+ DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_DEFAULT);
|
|
|
+ boolean compressionEnabled = conf.getBoolean(
|
|
|
+ DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY,
|
|
|
+ DFSConfigKeys.DFS_IMAGE_COMPRESS_DEFAULT);
|
|
|
+
|
|
|
+ if (loadInParallel) {
|
|
|
+ if (compressionEnabled) {
|
|
|
+ LOG.warn("Parallel Image loading and saving is not supported when {}" +
|
|
|
+ " is set to true. Parallel will be disabled.",
|
|
|
+ DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY);
|
|
|
+ loadInParallel = false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return loadInParallel;
|
|
|
+ }
|
|
|
+
|
|
|
public static final class Saver {
|
|
|
public static final int CHECK_CANCEL_INTERVAL = 4096;
|
|
|
+ private boolean writeSubSections = false;
|
|
|
+ private int inodesPerSubSection = Integer.MAX_VALUE;
|
|
|
|
|
|
private final SaveNamespaceContext context;
|
|
|
private final SaverContext saverContext;
|
|
|
private long currentOffset = FSImageUtil.MAGIC_HEADER.length;
|
|
|
+ private long subSectionOffset = currentOffset;
|
|
|
private MD5Hash savedDigest;
|
|
|
|
|
|
private FileChannel fileChannel;
|
|
@@ -463,10 +609,12 @@ public final class FSImageFormatProtobuf {
|
|
|
private OutputStream sectionOutputStream;
|
|
|
private CompressionCodec codec;
|
|
|
private OutputStream underlyingOutputStream;
|
|
|
+ private Configuration conf;
|
|
|
|
|
|
- Saver(SaveNamespaceContext context) {
|
|
|
+ Saver(SaveNamespaceContext context, Configuration conf) {
|
|
|
this.context = context;
|
|
|
this.saverContext = new SaverContext();
|
|
|
+ this.conf = conf;
|
|
|
}
|
|
|
|
|
|
public MD5Hash getSavedDigest() {
|
|
@@ -481,6 +629,29 @@ public final class FSImageFormatProtobuf {
|
|
|
return saverContext;
|
|
|
}
|
|
|
|
|
|
+ public int getInodesPerSubSection() {
|
|
|
+ return inodesPerSubSection;
|
|
|
+ }
|
|
|
+
|
|
|
+ public boolean shouldWriteSubSections() {
|
|
|
+ return writeSubSections;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Commit the length and offset of a fsimage section to the summary index,
|
|
|
+ * including the sub section, which will be committed before the section is
|
|
|
+ * committed.
|
|
|
+ * @param summary The image summary object
|
|
|
+ * @param name The name of the section to commit
|
|
|
+ * @param subSectionName The name of the sub-section to commit
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ public void commitSectionAndSubSection(FileSummary.Builder summary,
|
|
|
+ SectionName name, SectionName subSectionName) throws IOException {
|
|
|
+ commitSubSection(summary, subSectionName);
|
|
|
+ commitSection(summary, name);
|
|
|
+ }
|
|
|
+
|
|
|
public void commitSection(FileSummary.Builder summary, SectionName name)
|
|
|
throws IOException {
|
|
|
long oldOffset = currentOffset;
|
|
@@ -495,6 +666,35 @@ public final class FSImageFormatProtobuf {
|
|
|
summary.addSections(FileSummary.Section.newBuilder().setName(name.name)
|
|
|
.setLength(length).setOffset(currentOffset));
|
|
|
currentOffset += length;
|
|
|
+ subSectionOffset = currentOffset;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Commit the length and offset of a fsimage sub-section to the summary
|
|
|
+ * index.
|
|
|
+ * @param summary The image summary object
|
|
|
+ * @param name The name of the sub-section to commit
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ public void commitSubSection(FileSummary.Builder summary, SectionName name)
|
|
|
+ throws IOException {
|
|
|
+ if (!writeSubSections) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ LOG.debug("Saving a subsection for {}", name.toString());
|
|
|
+ // The output stream must be flushed before the length is obtained
|
|
|
+ // as the flush can move the length forward.
|
|
|
+ sectionOutputStream.flush();
|
|
|
+ long length = fileChannel.position() - subSectionOffset;
|
|
|
+ if (length == 0) {
|
|
|
+ LOG.warn("The requested section for {} is empty. It will not be " +
|
|
|
+ "output to the image", name.toString());
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ summary.addSections(FileSummary.Section.newBuilder().setName(name.name)
|
|
|
+ .setLength(length).setOffset(subSectionOffset));
|
|
|
+ subSectionOffset += length;
|
|
|
}
|
|
|
|
|
|
private void flushSectionOutputStream() throws IOException {
|
|
@@ -509,6 +709,7 @@ public final class FSImageFormatProtobuf {
|
|
|
* @throws IOException on fatal error.
|
|
|
*/
|
|
|
long save(File file, FSImageCompression compression) throws IOException {
|
|
|
+ enableSubSectionsIfRequired();
|
|
|
FileOutputStream fout = new FileOutputStream(file);
|
|
|
fileChannel = fout.getChannel();
|
|
|
try {
|
|
@@ -525,6 +726,47 @@ public final class FSImageFormatProtobuf {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private void enableSubSectionsIfRequired() {
|
|
|
+ boolean parallelEnabled = enableParallelSaveAndLoad(conf);
|
|
|
+ int inodeThreshold = conf.getInt(
|
|
|
+ DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_KEY,
|
|
|
+ DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_DEFAULT);
|
|
|
+ int targetSections = conf.getInt(
|
|
|
+ DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_KEY,
|
|
|
+ DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_DEFAULT);
|
|
|
+
|
|
|
+ if (parallelEnabled) {
|
|
|
+ if (targetSections <= 0) {
|
|
|
+ LOG.warn("{} is set to {}. It must be greater than zero. Setting to" +
|
|
|
+ " default of {}",
|
|
|
+ DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_KEY,
|
|
|
+ targetSections,
|
|
|
+ DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_DEFAULT);
|
|
|
+ targetSections =
|
|
|
+ DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_DEFAULT;
|
|
|
+ }
|
|
|
+ if (inodeThreshold <= 0) {
|
|
|
+ LOG.warn("{} is set to {}. It must be greater than zero. Setting to" +
|
|
|
+ " default of {}",
|
|
|
+ DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_KEY,
|
|
|
+ inodeThreshold,
|
|
|
+ DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_DEFAULT);
|
|
|
+ inodeThreshold =
|
|
|
+ DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_DEFAULT;
|
|
|
+ }
|
|
|
+ int inodeCount = context.getSourceNamesystem().dir.getInodeMapSize();
|
|
|
+ // Only enable parallel sections if there are enough inodes
|
|
|
+ if (inodeCount >= inodeThreshold) {
|
|
|
+ writeSubSections = true;
|
|
|
+ // Calculate the inodes per section rounded up to the nearest int
|
|
|
+ inodesPerSubSection = (inodeCount + targetSections - 1) /
|
|
|
+ targetSections;
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ writeSubSections = false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private static void saveFileSummary(OutputStream out, FileSummary summary)
|
|
|
throws IOException {
|
|
|
summary.writeDelimitedTo(out);
|
|
@@ -737,11 +979,15 @@ public final class FSImageFormatProtobuf {
|
|
|
EXTENDED_ACL("EXTENDED_ACL"),
|
|
|
ERASURE_CODING("ERASURE_CODING"),
|
|
|
INODE("INODE"),
|
|
|
+ INODE_SUB("INODE_SUB"),
|
|
|
INODE_REFERENCE("INODE_REFERENCE"),
|
|
|
+ INODE_REFERENCE_SUB("INODE_REFERENCE_SUB"),
|
|
|
SNAPSHOT("SNAPSHOT"),
|
|
|
INODE_DIR("INODE_DIR"),
|
|
|
+ INODE_DIR_SUB("INODE_DIR_SUB"),
|
|
|
FILES_UNDERCONSTRUCTION("FILES_UNDERCONSTRUCTION"),
|
|
|
SNAPSHOT_DIFF("SNAPSHOT_DIFF"),
|
|
|
+ SNAPSHOT_DIFF_SUB("SNAPSHOT_DIFF_SUB"),
|
|
|
SECRET_MANAGER("SECRET_MANAGER"),
|
|
|
CACHE_MANAGER("CACHE_MANAGER");
|
|
|
|