|
@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
|
|
|
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
|
|
|
import org.apache.hadoop.ipc.RPC;
|
|
|
import org.apache.hadoop.util.Daemon;
|
|
|
-import org.apache.hadoop.util.Time;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
@@ -98,6 +97,11 @@ public class JournalNodeSyncer {
|
|
|
|
|
|
void stopSync() {
|
|
|
shouldSync = false;
|
|
|
+ // Delete the edits.sync directory
|
|
|
+ File editsSyncDir = journal.getStorage().getEditsSyncDir();
|
|
|
+ if (editsSyncDir.exists()) {
|
|
|
+ FileUtil.fullyDelete(editsSyncDir);
|
|
|
+ }
|
|
|
if (syncJournalDaemon != null) {
|
|
|
syncJournalDaemon.interrupt();
|
|
|
}
|
|
@@ -112,6 +116,15 @@ public class JournalNodeSyncer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private boolean createEditsSyncDir() {
|
|
|
+ File editsSyncDir = journal.getStorage().getEditsSyncDir();
|
|
|
+ if (editsSyncDir.exists()) {
|
|
|
+ LOG.info(editsSyncDir + " directory already exists.");
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ return editsSyncDir.mkdir();
|
|
|
+ }
|
|
|
+
|
|
|
private boolean getOtherJournalNodeProxies() {
|
|
|
List<InetSocketAddress> otherJournalNodes = getOtherJournalNodeAddrs();
|
|
|
if (otherJournalNodes == null || otherJournalNodes.isEmpty()) {
|
|
@@ -135,35 +148,51 @@ public class JournalNodeSyncer {
|
|
|
}
|
|
|
|
|
|
private void startSyncJournalsDaemon() {
|
|
|
- syncJournalDaemon = new Daemon(new Runnable() {
|
|
|
- @Override
|
|
|
- public void run() {
|
|
|
- while(shouldSync) {
|
|
|
- try {
|
|
|
- if (!journal.isFormatted()) {
|
|
|
- LOG.warn("Journal not formatted. Cannot sync.");
|
|
|
+ syncJournalDaemon = new Daemon(() -> {
|
|
|
+ // Wait for journal to be formatted to create edits.sync directory
|
|
|
+ while(!journal.isFormatted()) {
|
|
|
+ try {
|
|
|
+ Thread.sleep(journalSyncInterval);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ LOG.error("JournalNodeSyncer daemon received Runtime exception.", e);
|
|
|
+ Thread.currentThread().interrupt();
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (!createEditsSyncDir()) {
|
|
|
+ LOG.error("Failed to create directory for downloading log " +
|
|
|
+ "segments: %s. Stopping Journal Node Sync.",
|
|
|
+ journal.getStorage().getEditsSyncDir());
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ while(shouldSync) {
|
|
|
+ try {
|
|
|
+ if (!journal.isFormatted()) {
|
|
|
+ LOG.warn("Journal cannot sync. Not formatted.");
|
|
|
+ } else {
|
|
|
+ syncJournals();
|
|
|
+ }
|
|
|
+ Thread.sleep(journalSyncInterval);
|
|
|
+ } catch (Throwable t) {
|
|
|
+ if (!shouldSync) {
|
|
|
+ if (t instanceof InterruptedException) {
|
|
|
+ LOG.info("Stopping JournalNode Sync.");
|
|
|
+ Thread.currentThread().interrupt();
|
|
|
+ return;
|
|
|
} else {
|
|
|
- syncJournals();
|
|
|
+ LOG.warn("JournalNodeSyncer received an exception while " +
|
|
|
+ "shutting down.", t);
|
|
|
}
|
|
|
- Thread.sleep(journalSyncInterval);
|
|
|
- } catch (Throwable t) {
|
|
|
- if (!shouldSync) {
|
|
|
- if (t instanceof InterruptedException) {
|
|
|
- LOG.info("Stopping JournalNode Sync.");
|
|
|
- } else {
|
|
|
- LOG.warn("JournalNodeSyncer received an exception while " +
|
|
|
- "shutting down.", t);
|
|
|
- }
|
|
|
- break;
|
|
|
- } else {
|
|
|
- if (t instanceof InterruptedException) {
|
|
|
- LOG.warn("JournalNodeSyncer interrupted", t);
|
|
|
- break;
|
|
|
- }
|
|
|
+ break;
|
|
|
+ } else {
|
|
|
+ if (t instanceof InterruptedException) {
|
|
|
+ LOG.warn("JournalNodeSyncer interrupted", t);
|
|
|
+ Thread.currentThread().interrupt();
|
|
|
+ return;
|
|
|
}
|
|
|
- LOG.error(
|
|
|
- "JournalNodeSyncer daemon received Runtime exception. ", t);
|
|
|
}
|
|
|
+ LOG.error(
|
|
|
+ "JournalNodeSyncer daemon received Runtime exception. ", t);
|
|
|
}
|
|
|
}
|
|
|
});
|
|
@@ -335,8 +364,8 @@ public class JournalNodeSyncer {
|
|
|
/**
|
|
|
* Transfer an edit log from one journal node to another for sync-up.
|
|
|
*/
|
|
|
- private boolean downloadMissingLogSegment(URL url, RemoteEditLog log) throws
|
|
|
- IOException {
|
|
|
+ private boolean downloadMissingLogSegment(URL url, RemoteEditLog log)
|
|
|
+ throws IOException {
|
|
|
LOG.info("Downloading missing Edit Log from " + url + " to " + jnStorage
|
|
|
.getRoot());
|
|
|
|
|
@@ -350,9 +379,10 @@ public class JournalNodeSyncer {
|
|
|
return true;
|
|
|
}
|
|
|
|
|
|
- final long milliTime = Time.monotonicNow();
|
|
|
- File tmpEditsFile = jnStorage.getTemporaryEditsFile(log.getStartTxId(), log
|
|
|
- .getEndTxId(), milliTime);
|
|
|
+ // Download the log segment to current.tmp directory first.
|
|
|
+ File tmpEditsFile = jnStorage.getTemporaryEditsFile(
|
|
|
+ log.getStartTxId(), log.getEndTxId());
|
|
|
+
|
|
|
try {
|
|
|
Util.doGetUrl(url, ImmutableList.of(tmpEditsFile), jnStorage, false,
|
|
|
logSegmentTransferTimeout, throttler);
|
|
@@ -367,14 +397,12 @@ public class JournalNodeSyncer {
|
|
|
LOG.info("Downloaded file " + tmpEditsFile.getName() + " of size " +
|
|
|
tmpEditsFile.length() + " bytes.");
|
|
|
|
|
|
- LOG.debug("Renaming " + tmpEditsFile.getName() + " to "
|
|
|
- + finalEditsFile.getName());
|
|
|
- boolean renameSuccess = journal.renameTmpSegment(tmpEditsFile,
|
|
|
+ final boolean moveSuccess = journal.moveTmpSegmentToCurrent(tmpEditsFile,
|
|
|
finalEditsFile, log.getEndTxId());
|
|
|
- if (!renameSuccess) {
|
|
|
- //If rename is not successful, delete the tmpFile
|
|
|
- LOG.debug("Renaming unsuccessful. Deleting temporary file: "
|
|
|
- + tmpEditsFile);
|
|
|
+ if (!moveSuccess) {
|
|
|
+ // If move is not successful, delete the tmpFile
|
|
|
+ LOG.debug("Move to current directory unsuccessful. Deleting temporary " +
|
|
|
+ "file: " + tmpEditsFile);
|
|
|
if (!tmpEditsFile.delete()) {
|
|
|
LOG.warn("Deleting " + tmpEditsFile + " has failed");
|
|
|
}
|