|
@@ -18,6 +18,9 @@
|
|
|
package org.apache.hadoop.hdfs.qjournal.server;
|
|
|
|
|
|
import org.apache.hadoop.classification.VisibleForTesting;
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos;
|
|
|
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
|
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
|
|
|
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -79,6 +82,7 @@ public class JournalNodeSyncer {
|
|
|
private int numOtherJNs;
|
|
|
private int journalNodeIndexForSync = 0;
|
|
|
private final long journalSyncInterval;
|
|
|
+ private final boolean tryFormatting;
|
|
|
private final int logSegmentTransferTimeout;
|
|
|
private final DataTransferThrottler throttler;
|
|
|
private final JournalMetrics metrics;
|
|
@@ -98,6 +102,9 @@ public class JournalNodeSyncer {
|
|
|
logSegmentTransferTimeout = conf.getInt(
|
|
|
DFSConfigKeys.DFS_EDIT_LOG_TRANSFER_TIMEOUT_KEY,
|
|
|
DFSConfigKeys.DFS_EDIT_LOG_TRANSFER_TIMEOUT_DEFAULT);
|
|
|
+ tryFormatting = conf.getBoolean(
|
|
|
+ DFSConfigKeys.DFS_JOURNALNODE_ENABLE_SYNC_FORMAT_KEY,
|
|
|
+ DFSConfigKeys.DFS_JOURNALNODE_ENABLE_SYNC_FORMAT_DEFAULT);
|
|
|
throttler = getThrottler(conf);
|
|
|
metrics = journal.getMetrics();
|
|
|
journalSyncerStarted = false;
|
|
@@ -171,6 +178,8 @@ public class JournalNodeSyncer {
|
|
|
// Wait for journal to be formatted to create edits.sync directory
|
|
|
while(!journal.isFormatted()) {
|
|
|
try {
|
|
|
+ // Format the journal with namespace info from the other JNs if it is not formatted
|
|
|
+ formatWithSyncer();
|
|
|
Thread.sleep(journalSyncInterval);
|
|
|
} catch (InterruptedException e) {
|
|
|
LOG.error("JournalNodeSyncer daemon received Runtime exception.", e);
|
|
@@ -187,7 +196,15 @@ public class JournalNodeSyncer {
|
|
|
while(shouldSync) {
|
|
|
try {
|
|
|
if (!journal.isFormatted()) {
|
|
|
- LOG.warn("Journal cannot sync. Not formatted.");
|
|
|
+ LOG.warn("Journal cannot sync. Not formatted. Trying to format with the syncer");
|
|
|
+ formatWithSyncer();
|
|
|
+ if (journal.isFormatted() && !createEditsSyncDir()) {
|
|
|
+ LOG.error("Failed to create directory for downloading log " +
|
|
|
+ "segments: {}. Stopping Journal Node Sync.",
|
|
|
+ journal.getStorage().getEditsSyncDir());
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ continue;
|
|
|
} else {
|
|
|
syncJournals();
|
|
|
}
|
|
@@ -233,6 +250,68 @@ public class JournalNodeSyncer {
|
|
|
journalNodeIndexForSync = (journalNodeIndexForSync + 1) % numOtherJNs;
|
|
|
}
|
|
|
|
|
|
+ private void formatWithSyncer() {
|
|
|
+ if (!tryFormatting) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ LOG.info("Trying to format the journal with the syncer");
|
|
|
+ try {
|
|
|
+ StorageInfo storage = null;
|
|
|
+ for (JournalNodeProxy jnProxy : otherJNProxies) {
|
|
|
+ if (!hasEditLogs(jnProxy)) {
|
|
|
+ // This avoids a race condition between `hdfs namenode -format` and
|
|
|
+ // JN syncer by checking if the other JN is not newly formatted.
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ HdfsServerProtos.StorageInfoProto storageInfoResponse =
|
|
|
+ jnProxy.jnProxy.getStorageInfo(jid, nameServiceId);
|
|
|
+ storage = PBHelper.convert(
|
|
|
+ storageInfoResponse, HdfsServerConstants.NodeType.JOURNAL_NODE
|
|
|
+ );
|
|
|
+ if (storage.getNamespaceID() == 0) {
|
|
|
+ LOG.error("Got invalid StorageInfo from " + jnProxy);
|
|
|
+ storage = null;
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ LOG.info("Got StorageInfo " + storage + " from " + jnProxy);
|
|
|
+ break;
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.error("Could not get StorageInfo from " + jnProxy, e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (storage == null) {
|
|
|
+ LOG.error("Could not get StorageInfo from any JournalNode. " +
|
|
|
+ "JournalNodeSyncer cannot format the journal.");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ NamespaceInfo nsInfo = new NamespaceInfo(storage);
|
|
|
+ journal.format(nsInfo, true);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.error("Exception in formatting the journal with the syncer", e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private boolean hasEditLogs(JournalNodeProxy journalProxy) {
|
|
|
+ GetEditLogManifestResponseProto editLogManifest;
|
|
|
+ try {
|
|
|
+ editLogManifest = journalProxy.jnProxy.getEditLogManifestFromJournal(
|
|
|
+ jid, nameServiceId, 0, false);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.error("Could not get edit log manifest from " + journalProxy, e);
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ List<RemoteEditLog> otherJournalEditLogs = PBHelper.convert(
|
|
|
+ editLogManifest.getManifest()).getLogs();
|
|
|
+ if (otherJournalEditLogs == null || otherJournalEditLogs.isEmpty()) {
|
|
|
+ LOG.warn("Journal at " + journalProxy.jnAddr + " has no edit logs");
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
private void syncWithJournalAtIndex(int index) {
|
|
|
LOG.info("Syncing Journal " + jn.getBoundIpcAddress().getAddress() + ":"
|
|
|
+ jn.getBoundIpcAddress().getPort() + " with "
|