private void startSyncer(Journal journal, String jid, String nameServiceId) { JournalNodeSyncer jSyncer = journalSyncersById.get(jid); if (jSyncer == null) { jSyncer = new JournalNodeSyncer(this, journal, jid, conf, nameServiceId); journalSyncersById.put(jid, jSyncer); } jSyncer.start(nameServiceId); }
private boolean getOtherJournalNodeProxies() { List<InetSocketAddress> otherJournalNodes = getOtherJournalNodeAddrs(); if (otherJournalNodes == null || otherJournalNodes.isEmpty()) { LOG.warn("Other JournalNode addresses not available. Journal Syncing " + "cannot be done"); return false; } for (InetSocketAddress addr : otherJournalNodes) { try { otherJNProxies.add(new JournalNodeProxy(addr)); } catch (IOException e) { LOG.warn("Could not add proxy for Journal at addresss " + addr, e); } } if (otherJNProxies.isEmpty()) { LOG.error("Cannot sync as there is no other JN available for sync."); return false; } numOtherJNs = otherJNProxies.size(); return true; }
public void start(String nsId) { if (nsId != null) { this.nameServiceId = nsId; journal.setTriedJournalSyncerStartedwithnsId(true); } if (!journalSyncerStarted && getOtherJournalNodeProxies()) { LOG.info("Starting SyncJournal daemon for journal " + jid); startSyncJournalsDaemon(); journalSyncerStarted = true; } }
return; List<RemoteEditLog> missingLogs = getMissingLogList(thisJournalEditLogs, otherJournalEditLogs); if (remoteJNproxy.httpServerUrl == null) { if (response.hasFromURL()) { remoteJNproxy.httpServerUrl = getHttpServerURI( response.getFromURL(), remoteJNproxy.jnAddr.getHostName()); } else { .getStartTxId(), nsInfo, false); url = new URL(remoteJNproxy.httpServerUrl, urlPath); success = downloadMissingLogSegment(url, missingLog); } catch (URISyntaxException e) { LOG.error("EditLogManifest's fromUrl field syntax incorrect", e);
if (!createEditsSyncDir()) { LOG.error("Failed to create directory for downloading log " + "segments: %s. Stopping Journal Node Sync.", LOG.warn("Journal cannot sync. Not formatted."); } else { syncJournals();
JournalNodeSyncer(JournalNode jouranlNode, Journal journal, String jid, Configuration conf, String nameServiceId) { this.jn = jouranlNode; this.journal = journal; this.jid = jid; this.nameServiceId = nameServiceId; this.jnStorage = journal.getStorage(); this.conf = conf; journalSyncInterval = conf.getLong( DFSConfigKeys.DFS_JOURNALNODE_SYNC_INTERVAL_KEY, DFSConfigKeys.DFS_JOURNALNODE_SYNC_INTERVAL_DEFAULT); logSegmentTransferTimeout = conf.getInt( DFSConfigKeys.DFS_EDIT_LOG_TRANSFER_TIMEOUT_KEY, DFSConfigKeys.DFS_EDIT_LOG_TRANSFER_TIMEOUT_DEFAULT); throttler = getThrottler(conf); metrics = journal.getMetrics(); journalSyncerStarted = false; }
private void syncWithJournalAtIndex(int index) { LOG.info("Syncing Journal " + jn.getBoundIpcAddress().getAddress() + ":" + jn.getBoundIpcAddress().getPort() + " with " + otherJNProxies.get(index) + ", journal id: " + jid); final InterQJournalProtocol jnProxy = otherJNProxies.get(index).jnProxy; if (jnProxy == null) { LOG.error("JournalNode Proxy not found."); return; } List<RemoteEditLog> thisJournalEditLogs; try { thisJournalEditLogs = journal.getEditLogManifest(0, false).getLogs(); } catch (IOException e) { LOG.error("Exception in getting local edit log manifest", e); return; } GetEditLogManifestResponseProto editLogManifest; try { editLogManifest = jnProxy.getEditLogManifestFromJournal(jid, nameServiceId, 0, false); } catch (IOException e) { LOG.error("Could not sync with Journal at " + otherJNProxies.get(journalNodeIndexForSync), e); return; } getMissingLogSegments(thisJournalEditLogs, editLogManifest, otherJNProxies.get(index)); }
return null; } else { return getJournalAddrList(uriStr);