|
@@ -19,7 +19,11 @@ package org.apache.hadoop.hdfs.qjournal.server;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
+import java.util.List;
|
|
|
+import java.util.regex.Matcher;
|
|
|
+import java.util.regex.Pattern;
|
|
|
|
|
|
+import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
|
|
import org.apache.hadoop.hdfs.server.common.Storage;
|
|
@@ -28,6 +32,8 @@ import org.apache.hadoop.hdfs.server.namenode.FileJournalManager;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
|
|
|
|
+import com.google.common.collect.ImmutableList;
|
|
|
+
|
|
|
/**
|
|
|
* A {@link Storage} implementation for the {@link JournalNode}.
|
|
|
*
|
|
@@ -39,6 +45,15 @@ class JNStorage extends Storage {
|
|
|
private final FileJournalManager fjm;
|
|
|
private final StorageDirectory sd;
|
|
|
private StorageState state;
|
|
|
+
|
|
|
+
|
|
|
+ private static final List<Pattern> CURRENT_DIR_PURGE_REGEXES =
|
|
|
+ ImmutableList.of(
|
|
|
+ Pattern.compile("edits_\\d+-(\\d+)"),
|
|
|
+ Pattern.compile("edits_inprogress_(\\d+)(?:\\..*)?"));
|
|
|
+
|
|
|
+ private static final List<Pattern> PAXOS_DIR_PURGE_REGEXES =
|
|
|
+ ImmutableList.of(Pattern.compile("(\\d+)"));
|
|
|
|
|
|
/**
|
|
|
* @param logDir the path to the directory in which data will be stored
|
|
@@ -111,6 +126,48 @@ class JNStorage extends Storage {
|
|
|
File getPaxosDir() {
|
|
|
return new File(sd.getCurrentDir(), "paxos");
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Remove any log files and associated paxos files which are older than
|
|
|
+ * the given txid.
|
|
|
+ */
|
|
|
+ void purgeDataOlderThan(long minTxIdToKeep) throws IOException {
|
|
|
+ purgeMatching(sd.getCurrentDir(),
|
|
|
+ CURRENT_DIR_PURGE_REGEXES, minTxIdToKeep);
|
|
|
+ purgeMatching(getPaxosDir(), PAXOS_DIR_PURGE_REGEXES, minTxIdToKeep);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Purge files in the given directory which match any of the set of patterns.
|
|
|
+ * The patterns must have a single numeric capture group which determines
|
|
|
+ * the associated transaction ID of the file. Only those files for which
|
|
|
+ * the transaction ID is less than the <code>minTxIdToKeep</code> parameter
|
|
|
+ * are removed.
|
|
|
+ */
|
|
|
+ private static void purgeMatching(File dir, List<Pattern> patterns,
|
|
|
+ long minTxIdToKeep) throws IOException {
|
|
|
+
|
|
|
+ for (File f : FileUtil.listFiles(dir)) {
|
|
|
+ if (!f.isFile()) continue;
|
|
|
+
|
|
|
+ for (Pattern p : patterns) {
|
|
|
+ Matcher matcher = p.matcher(f.getName());
|
|
|
+ if (matcher.matches()) {
|
|
|
+ // This parsing will always succeed since the group(1) is
|
|
|
+ // /\d+/ in the regex itself.
|
|
|
+ long txid = Long.valueOf(matcher.group(1));
|
|
|
+ if (txid < minTxIdToKeep) {
|
|
|
+ LOG.info("Purging no-longer needed file " + txid);
|
|
|
+ if (!f.delete()) {
|
|
|
+ LOG.warn("Unable to delete no-longer-needed data " +
|
|
|
+ f);
|
|
|
+ }
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
void format(NamespaceInfo nsInfo) throws IOException {
|
|
|
setStorageInfo(nsInfo);
|